1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258
3259
3260
3261
3262
3263
3264
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274
3275
3276
3277
3278
3279
3280
3281
3282
3283
3284
3285
3286
3287
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311
3312
3313
3314
3315
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363
3364
3365
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375
3376
3377
3378
3379
3380
3381
3382
3383
3384
3385
3386
3387
3388
3389
3390
3391
3392
3393
3394
3395
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405
3406
3407
3408
3409
3410
3411
3412
3413
3414
3415
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
3426
3427
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442
3443
3444
3445
3446
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458
3459
3460
3461
3462
3463
3464
3465
3466
3467
3468
3469
3470
3471
3472
3473
3474
3475
3476
3477
3478
3479
3480
3481
3482
3483
3484
3485
3486
3487
3488
3489
3490
3491
3492
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516
3517
3518
3519
3520
3521
3522
3523
3524
3525
3526
3527
3528
3529
3530
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548
3549
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599
3600
3601
3602
3603
3604
3605
3606
3607
3608
3609
3610
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636
3637
3638
3639
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651
3652
3653
3654
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691
3692
3693
3694
3695
3696
3697
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711
3712
3713
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745
3746
3747
3748
3749
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
3767
3768
3769
3770
3771
3772
3773
3774
3775
3776
3777
3778
3779
3780
3781
3782
3783
3784
3785
3786
3787
3788
3789
3790
3791
3792
3793
3794
3795
3796
3797
3798
3799
3800
3801
3802
3803
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818
3819
3820
3821
3822
3823
3824
3825
3826
3827
3828
3829
3830
3831
3832
3833
3834
3835
3836
3837
3838
3839
3840
3841
3842
3843
3844
3845
3846
3847
3848
3849
3850
3851
3852
3853
3854
3855
3856
3857
3858
3859
3860
3861
3862
3863
3864
3865
3866
3867
3868
3869
3870
3871
3872
3873
3874
3875
3876
3877
3878
3879
3880
3881
3882
3883
3884
3885
3886
3887
3888
3889
3890
3891
3892
3893
3894
3895
3896
3897
3898
3899
3900
3901
3902
3903
3904
3905
3906
3907
3908
3909
3910
3911
3912
3913
3914
3915
3916
3917
3918
3919
3920
3921
3922
3923
3924
3925
3926
3927
3928
3929
3930
3931
3932
3933
3934
3935
3936
3937
3938
3939
3940
3941
3942
3943
3944
3945
3946
3947
3948
3949
3950
3951
3952
3953
3954
3955
3956
3957
3958
3959
3960
3961
3962
3963
3964
3965
3966
3967
3968
3969
3970
3971
3972
3973
3974
3975
3976
3977
3978
3979
3980
3981
3982
3983
3984
3985
3986
3987
3988
3989
3990
3991
3992
3993
3994
3995
3996
3997
3998
3999
4000
4001
4002
4003
4004
4005
4006
4007
4008
4009
4010
4011
4012
4013
4014
4015
4016
4017
4018
4019
4020
4021
4022
4023
4024
4025
4026
4027
4028
4029
4030
4031
4032
4033
4034
4035
4036
4037
4038
4039
4040
4041
4042
4043
4044
4045
4046
4047
4048
4049
4050
4051
4052
4053
4054
4055
4056
4057
4058
4059
4060
4061
4062
4063
4064
4065
4066
4067
4068
4069
4070
4071
4072
4073
4074
4075
4076
4077
4078
4079
4080
4081
4082
4083
4084
4085
4086
4087
4088
4089
4090
4091
4092
4093
4094
4095
4096
4097
4098
4099
4100
4101
4102
4103
4104
4105
4106
4107
4108
4109
4110
4111
4112
4113
4114
4115
4116
4117
4118
4119
4120
4121
4122
4123
4124
4125
4126
4127
4128
4129
4130
4131
4132
4133
4134
4135
4136
4137
4138
4139
4140
4141
4142
4143
4144
4145
4146
4147
4148
4149
4150
4151
4152
4153
4154
4155
4156
4157
4158
4159
4160
4161
4162
4163
4164
4165
4166
4167
4168
4169
4170
4171
4172
4173
4174
4175
4176
4177
4178
4179
4180
4181
4182
4183
4184
4185
4186
4187
4188
4189
4190
4191
4192
4193
4194
4195
4196
4197
4198
4199
4200
4201
4202
4203
4204
4205
4206
4207
4208
4209
4210
4211
4212
4213
4214
4215
4216
4217
4218
4219
4220
4221
4222
4223
4224
4225
4226
4227
4228
4229
4230
4231
4232
4233
4234
4235
4236
4237
4238
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248
4249
4250
4251
4252
4253
4254
4255
4256
4257
4258
4259
4260
4261
4262
4263
4264
4265
4266
4267
4268
4269
4270
4271
4272
4273
4274
4275
4276
4277
4278
4279
4280
4281
4282
4283
4284
4285
4286
4287
4288
4289
4290
4291
4292
4293
4294
4295
4296
4297
4298
4299
4300
4301
4302
4303
4304
4305
4306
4307
4308
4309
4310
4311
4312
4313
4314
4315
4316
4317
4318
4319
4320
4321
4322
4323
4324
4325
4326
4327
4328
4329
4330
4331
4332
4333
4334
4335
4336
4337
4338
4339
4340
4341
4342
4343
4344
4345
4346
4347
4348
4349
4350
4351
4352
4353
4354
4355
4356
4357
4358
4359
4360
4361
4362
4363
4364
4365
4366
4367
4368
4369
4370
4371
4372
4373
4374
4375
4376
4377
4378
4379
4380
4381
4382
4383
4384
4385
4386
4387
4388
4389
4390
4391
4392
4393
4394
4395
4396
4397
4398
4399
4400
4401
4402
4403
4404
4405
4406
4407
4408
4409
4410
4411
4412
4413
4414
4415
4416
4417
4418
4419
4420
4421
4422
4423
4424
4425
4426
4427
4428
4429
4430
4431
4432
4433
4434
4435
4436
4437
4438
4439
4440
4441
4442
4443
4444
4445
4446
4447
4448
4449
4450
4451
4452
4453
4454
4455
4456
4457
4458
4459
4460
4461
4462
4463
4464
4465
4466
4467
4468
4469
4470
4471
4472
4473
4474
4475
4476
4477
4478
4479
4480
4481
4482
4483
4484
4485
4486
4487
4488
4489
4490
4491
4492
4493
4494
4495
4496
4497
4498
4499
4500
4501
4502
4503
4504
4505
4506
4507
4508
4509
4510
4511
4512
4513
4514
4515
4516
4517
4518
4519
4520
4521
4522
4523
4524
4525
4526
4527
4528
4529
4530
4531
4532
4533
4534
4535
4536
4537
4538
4539
4540
4541
4542
4543
4544
4545
4546
4547
4548
4549
4550
4551
4552
4553
4554
4555
4556
4557
4558
4559
4560
4561
4562
4563
4564
4565
4566
4567
4568
4569
4570
4571
4572
4573
4574
4575
4576
4577
4578
4579
4580
4581
4582
4583
4584
4585
4586
4587
4588
4589
4590
4591
4592
4593
4594
4595
4596
4597
4598
4599
4600
4601
4602
4603
4604
4605
4606
4607
4608
4609
4610
4611
4612
4613
4614
4615
4616
4617
4618
4619
4620
4621
4622
4623
4624
4625
4626
4627
4628
4629
4630
4631
4632
4633
4634
4635
4636
4637
4638
4639
4640
4641
4642
4643
4644
4645
4646
4647
4648
4649
4650
4651
4652
4653
4654
4655
4656
4657
4658
4659
4660
4661
4662
4663
4664
4665
4666
4667
4668
4669
4670
4671
4672
4673
4674
4675
4676
4677
4678
4679
4680
4681
4682
4683
4684
4685
4686
4687
4688
4689
4690
4691
4692
4693
4694
4695
4696
4697
4698
4699
4700
4701
4702
4703
4704
4705
4706
4707
4708
4709
4710
4711
4712
4713
4714
4715
4716
4717
4718
4719
4720
4721
4722
4723
4724
4725
4726
4727
4728
4729
4730
4731
4732
4733
4734
4735
4736
4737
4738
4739
4740
4741
4742
4743
4744
4745
4746
4747
4748
4749
4750
4751
4752
4753
4754
4755
4756
4757
4758
4759
4760
4761
4762
4763
4764
4765
4766
4767
4768
4769
4770
4771
4772
4773
4774
4775
4776
4777
4778
4779
4780
4781
4782
4783
4784
4785
4786
4787
4788
4789
4790
4791
4792
4793
4794
4795
4796
4797
4798
4799
4800
4801
4802
4803
4804
4805
4806
4807
4808
4809
4810
4811
4812
4813
4814
4815
4816
4817
4818
4819
4820
4821
4822
4823
4824
4825
4826
4827
4828
4829
4830
4831
4832
4833
4834
4835
4836
4837
4838
4839
4840
4841
4842
4843
4844
4845
4846
4847
4848
4849
4850
4851
4852
4853
4854
4855
4856
4857
4858
4859
4860
4861
4862
4863
4864
4865
4866
4867
4868
4869
4870
4871
4872
4873
4874
4875
4876
4877
4878
4879
4880
4881
4882
4883
4884
4885
4886
4887
4888
4889
4890
4891
4892
4893
4894
4895
4896
4897
4898
4899
4900
4901
4902
4903
4904
4905
4906
4907
4908
4909
4910
4911
4912
4913
4914
4915
4916
4917
4918
4919
4920
4921
4922
4923
4924
4925
4926
4927
4928
4929
4930
4931
4932
4933
4934
4935
4936
4937
4938
4939
4940
4941
4942
4943
4944
4945
4946
4947
4948
4949
4950
4951
4952
4953
4954
4955
4956
4957
4958
4959
4960
4961
4962
4963
4964
4965
4966
4967
4968
4969
4970
4971
4972
4973
4974
4975
4976
4977
4978
4979
4980
4981
4982
4983
4984
4985
4986
4987
4988
4989
4990
4991
4992
4993
4994
4995
4996
4997
4998
4999
5000
5001
5002
5003
5004
5005
5006
5007
5008
5009
5010
5011
5012
5013
5014
5015
5016
5017
5018
5019
5020
5021
5022
5023
5024
5025
5026
5027
5028
5029
5030
5031
5032
5033
5034
5035
5036
5037
5038
5039
5040
5041
5042
5043
5044
5045
5046
5047
5048
5049
5050
5051
5052
5053
5054
5055
5056
5057
5058
5059
5060
5061
5062
5063
5064
5065
5066
5067
5068
5069
5070
5071
5072
5073
5074
5075
5076
5077
5078
5079
5080
5081
5082
5083
5084
5085
5086
5087
5088
5089
5090
5091
5092
5093
5094
5095
5096
5097
5098
5099
5100
5101
5102
5103
5104
5105
5106
5107
5108
5109
5110
5111
5112
5113
5114
5115
5116
5117
5118
5119
5120
5121
5122
5123
5124
5125
5126
5127
5128
5129
5130
5131
5132
5133
5134
5135
5136
5137
5138
5139
5140
5141
5142
5143
5144
5145
5146
5147
5148
5149
5150
5151
5152
5153
5154
5155
5156
5157
5158
5159
5160
5161
5162
5163
5164
5165
5166
5167
5168
5169
5170
5171
5172
5173
5174
5175
5176
5177
5178
5179
5180
5181
5182
5183
5184
5185
5186
5187
5188
5189
5190
5191
5192
5193
5194
5195
5196
5197
5198
5199
5200
5201
5202
5203
5204
5205
5206
5207
5208
5209
5210
5211
5212
5213
5214
5215
5216
5217
5218
5219
5220
5221
5222
5223
5224
5225
5226
5227
5228
5229
5230
5231
5232
5233
5234
5235
5236
5237
5238
5239
5240
5241
5242
5243
5244
5245
5246
5247
5248
5249
5250
5251
5252
5253
5254
5255
5256
5257
5258
5259
5260
5261
5262
5263
5264
5265
5266
5267
5268
5269
5270
5271
5272
5273
5274
5275
5276
5277
5278
5279
5280
5281
5282
5283
5284
5285
5286
5287
5288
5289
5290
5291
5292
5293
5294
5295
5296
5297
5298
5299
5300
5301
5302
5303
5304
5305
5306
5307
5308
5309
5310
5311
5312
5313
5314
5315
5316
5317
5318
5319
5320
5321
5322
5323
5324
5325
5326
5327
5328
5329
5330
5331
5332
5333
5334
5335
5336
5337
5338
5339
5340
5341
5342
5343
5344
5345
5346
5347
5348
5349
5350
5351
5352
5353
5354
5355
5356
5357
5358
5359
5360
5361
5362
5363
5364
5365
5366
5367
5368
5369
5370
5371
5372
5373
5374
5375
5376
5377
5378
5379
5380
5381
5382
5383
5384
5385
5386
5387
5388
5389
5390
5391
5392
5393
5394
5395
5396
5397
5398
5399
5400
5401
5402
5403
5404
5405
5406
5407
5408
5409
5410
5411
5412
5413
5414
5415
5416
5417
5418
5419
5420
5421
5422
5423
5424
5425
5426
5427
5428
5429
5430
5431
5432
5433
5434
5435
5436
5437
5438
5439
5440
5441
5442
5443
5444
5445
5446
5447
5448
5449
5450
5451
5452
5453
5454
5455
5456
5457
5458
5459
5460
5461
5462
5463
5464
5465
5466
5467
5468
5469
5470
5471
5472
5473
5474
5475
5476
5477
5478
5479
5480
5481
5482
5483
5484
5485
5486
5487
5488
5489
5490
5491
5492
5493
5494
5495
5496
5497
5498
5499
5500
5501
5502
5503
5504
5505
5506
5507
5508
5509
5510
5511
5512
5513
5514
5515
5516
5517
5518
5519
5520
5521
5522
5523
5524
5525
5526
5527
5528
5529
5530
5531
5532
5533
5534
5535
5536
5537
5538
5539
5540
5541
5542
5543
5544
5545
5546
5547
5548
5549
5550
5551
5552
5553
5554
5555
5556
5557
5558
5559
5560
5561
5562
5563
5564
5565
5566
5567
5568
5569
5570
5571
5572
5573
5574
5575
5576
5577
5578
5579
5580
5581
5582
5583
5584
5585
5586
5587
5588
5589
5590
5591
5592
5593
5594
5595
5596
5597
5598
5599
5600
5601
5602
5603
5604
5605
5606
5607
5608
5609
5610
5611
5612
5613
5614
5615
5616
5617
5618
5619
5620
5621
5622
5623
5624
5625
5626
5627
5628
5629
5630
5631
5632
5633
5634
5635
5636
5637
5638
5639
5640
5641
5642
5643
5644
5645
5646
5647
5648
5649
5650
5651
5652
5653
5654
5655
5656
5657
5658
5659
5660
5661
5662
5663
5664
5665
5666
5667
5668
5669
5670
5671
5672
5673
5674
5675
5676
5677
5678
5679
5680
5681
5682
5683
5684
5685
5686
5687
5688
5689
5690
5691
5692
5693
5694
5695
5696
5697
5698
5699
5700
5701
5702
5703
5704
5705
5706
5707
5708
5709
5710
5711
5712
5713
5714
5715
5716
5717
5718
5719
5720
5721
5722
5723
5724
5725
5726
5727
5728
5729
5730
5731
5732
5733
5734
5735
5736
5737
5738
5739
5740
5741
5742
5743
5744
5745
5746
5747
5748
5749
5750
5751
5752
5753
5754
5755
5756
5757
5758
5759
5760
5761
5762
5763
5764
5765
5766
5767
5768
5769
5770
5771
5772
5773
5774
5775
5776
5777
5778
5779
5780
5781
5782
5783
5784
5785
5786
5787
5788
5789
5790
5791
5792
5793
5794
5795
5796
5797
5798
5799
5800
5801
5802
5803
5804
5805
5806
5807
5808
5809
5810
5811
5812
5813
5814
5815
5816
5817
5818
5819
5820
5821
5822
5823
5824
5825
5826
5827
5828
5829
5830
5831
5832
5833
5834
5835
5836
5837
5838
5839
5840
5841
5842
5843
5844
5845
5846
5847
5848
5849
5850
5851
5852
5853
5854
5855
5856
5857
5858
5859
5860
5861
5862
5863
5864
5865
5866
5867
5868
5869
5870
5871
5872
5873
5874
5875
5876
5877
5878
5879
5880
5881
5882
5883
5884
5885
5886
5887
5888
5889
5890
5891
5892
5893
5894
5895
5896
5897
5898
5899
5900
5901
5902
5903
5904
5905
5906
5907
5908
5909
5910
5911
5912
5913
5914
5915
5916
5917
5918
5919
5920
5921
5922
5923
5924
5925
5926
5927
5928
5929
5930
5931
5932
5933
5934
5935
5936
5937
5938
5939
5940
5941
5942
5943
5944
5945
5946
5947
5948
5949
5950
5951
5952
5953
5954
5955
5956
5957
5958
5959
5960
5961
5962
5963
5964
5965
5966
5967
5968
5969
|
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2015, Magnus Edenhill
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include "rdkafka_int.h"
#include "rdkafka_broker.h"
#include "rdkafka_request.h"
#include "rdkafka_topic.h"
#include "rdkafka_partition.h"
#include "rdkafka_assignor.h"
#include "rdkafka_offset.h"
#include "rdkafka_metadata.h"
#include "rdkafka_cgrp.h"
#include "rdkafka_interceptor.h"
#include "rdmap.h"
#include "rdunittest.h"
#include <ctype.h>
#include <stdarg.h>
static void rd_kafka_cgrp_offset_commit_tmr_cb(rd_kafka_timers_t *rkts,
void *arg);
static rd_kafka_error_t *
rd_kafka_cgrp_assign(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *assignment);
static rd_kafka_error_t *rd_kafka_cgrp_unassign(rd_kafka_cgrp_t *rkcg);
static rd_kafka_error_t *
rd_kafka_cgrp_incremental_assign(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *partitions);
static rd_kafka_error_t *
rd_kafka_cgrp_incremental_unassign(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *partitions);
static rd_kafka_op_res_t rd_kafka_cgrp_op_serve(rd_kafka_t *rk,
rd_kafka_q_t *rkq,
rd_kafka_op_t *rko,
rd_kafka_q_cb_type_t cb_type,
void *opaque);
static void rd_kafka_cgrp_group_leader_reset(rd_kafka_cgrp_t *rkcg,
const char *reason);
static RD_INLINE int rd_kafka_cgrp_try_terminate(rd_kafka_cgrp_t *rkcg);
static void rd_kafka_cgrp_revoke_all_rejoin(rd_kafka_cgrp_t *rkcg,
rd_bool_t assignment_lost,
rd_bool_t initiating,
const char *reason);
static void rd_kafka_cgrp_revoke_all_rejoin_maybe(rd_kafka_cgrp_t *rkcg,
rd_bool_t assignment_lost,
rd_bool_t initiating,
const char *reason);
static void rd_kafka_cgrp_group_is_rebalancing(rd_kafka_cgrp_t *rkcg);
static void
rd_kafka_cgrp_max_poll_interval_check_tmr_cb(rd_kafka_timers_t *rkts,
void *arg);
static rd_kafka_resp_err_t
rd_kafka_cgrp_subscribe(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *rktparlist);
static void rd_kafka_cgrp_group_assignment_set(
rd_kafka_cgrp_t *rkcg,
const rd_kafka_topic_partition_list_t *partitions);
static void rd_kafka_cgrp_group_assignment_modify(
rd_kafka_cgrp_t *rkcg,
rd_bool_t add,
const rd_kafka_topic_partition_list_t *partitions);
static void
rd_kafka_cgrp_handle_assignment(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *assignment);
/**
* @returns true if the current assignment is lost.
*/
rd_bool_t rd_kafka_cgrp_assignment_is_lost(rd_kafka_cgrp_t *rkcg) {
return rd_atomic32_get(&rkcg->rkcg_assignment_lost) != 0;
}
/**
* @brief Call when the current assignment has been lost, with a
* human-readable reason.
*/
static void rd_kafka_cgrp_assignment_set_lost(rd_kafka_cgrp_t *rkcg,
char *fmt,
...) RD_FORMAT(printf, 2, 3);
static void
rd_kafka_cgrp_assignment_set_lost(rd_kafka_cgrp_t *rkcg, char *fmt, ...) {
va_list ap;
char reason[256];
if (!rkcg->rkcg_group_assignment)
return;
va_start(ap, fmt);
rd_vsnprintf(reason, sizeof(reason), fmt, ap);
va_end(ap);
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "LOST",
"Group \"%s\": "
"current assignment of %d partition(s) lost: %s",
rkcg->rkcg_group_id->str, rkcg->rkcg_group_assignment->cnt,
reason);
rd_atomic32_set(&rkcg->rkcg_assignment_lost, rd_true);
}
/**
* @brief Call when the current assignment is no longer considered lost, with a
* human-readable reason.
*/
static void
rd_kafka_cgrp_assignment_clear_lost(rd_kafka_cgrp_t *rkcg, char *fmt, ...) {
va_list ap;
char reason[256];
if (!rd_atomic32_get(&rkcg->rkcg_assignment_lost))
return;
va_start(ap, fmt);
rd_vsnprintf(reason, sizeof(reason), fmt, ap);
va_end(ap);
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "LOST",
"Group \"%s\": "
"current assignment no longer considered lost: %s",
rkcg->rkcg_group_id->str, reason);
rd_atomic32_set(&rkcg->rkcg_assignment_lost, rd_false);
}
/**
* @brief The rebalance protocol currently in use. This will be
* RD_KAFKA_REBALANCE_PROTOCOL_NONE if the consumer has not
* (yet) joined a group, else it will match the rebalance
* protocol of the configured assignor(s).
*
* @locality main thread
*/
rd_kafka_rebalance_protocol_t
rd_kafka_cgrp_rebalance_protocol(rd_kafka_cgrp_t *rkcg) {
if (!rkcg->rkcg_assignor)
return RD_KAFKA_REBALANCE_PROTOCOL_NONE;
return rkcg->rkcg_assignor->rkas_protocol;
}
/**
* @returns true if the cgrp is awaiting a protocol response. This prohibits
* the join-state machine to proceed before the current state
* is done.
*/
static rd_bool_t rd_kafka_cgrp_awaiting_response(rd_kafka_cgrp_t *rkcg) {
return rkcg->rkcg_wait_resp != -1;
}
/**
* @brief Set flag indicating we are waiting for a coordinator response
* for the given request.
*
* This is used for specific requests to postpone rejoining the group if
* there are outstanding JoinGroup or SyncGroup requests.
*
* @locality main thread
*/
static void rd_kafka_cgrp_set_wait_resp(rd_kafka_cgrp_t *rkcg, int16_t ApiKey) {
rd_assert(rkcg->rkcg_wait_resp == -1);
rkcg->rkcg_wait_resp = ApiKey;
}
/**
* @brief Clear the flag that says we're waiting for a coordinator response
* for the given \p request.
*
* @param request Original request, possibly NULL (for errors).
*
* @locality main thread
*/
static void rd_kafka_cgrp_clear_wait_resp(rd_kafka_cgrp_t *rkcg,
int16_t ApiKey) {
rd_assert(rkcg->rkcg_wait_resp == ApiKey);
rkcg->rkcg_wait_resp = -1;
}
/**
* @struct Auxillary glue type used for COOPERATIVE rebalance set operations.
*/
typedef struct PartitionMemberInfo_s {
const rd_kafka_group_member_t *member;
rd_bool_t members_match;
} PartitionMemberInfo_t;
static PartitionMemberInfo_t *
PartitionMemberInfo_new(const rd_kafka_group_member_t *member,
rd_bool_t members_match) {
PartitionMemberInfo_t *pmi;
pmi = rd_calloc(1, sizeof(*pmi));
pmi->member = member;
pmi->members_match = members_match;
return pmi;
}
static void PartitionMemberInfo_free(void *p) {
PartitionMemberInfo_t *pmi = p;
rd_free(pmi);
}
typedef RD_MAP_TYPE(const rd_kafka_topic_partition_t *,
PartitionMemberInfo_t *) map_toppar_member_info_t;
/**
* @returns true if consumer has joined the group and thus requires a leave.
*/
#define RD_KAFKA_CGRP_HAS_JOINED(rkcg) \
(rkcg->rkcg_member_id != NULL && \
RD_KAFKAP_STR_LEN((rkcg)->rkcg_member_id) > 0)
/**
* @returns true if cgrp is waiting for a rebalance_cb to be handled by
* the application.
*/
#define RD_KAFKA_CGRP_WAIT_ASSIGN_CALL(rkcg) \
((rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL)
/**
* @returns true if a rebalance is in progress.
*
* 1. In WAIT_JOIN or WAIT_METADATA state with a member-id set,
* this happens on rejoin.
* 2. In WAIT_SYNC waiting for the group to rebalance on the broker.
* 3. in *_WAIT_UNASSIGN_TO_COMPLETE waiting for unassigned partitions to
* stop fetching, et.al.
* 4. In _WAIT_*ASSIGN_CALL waiting for the application to handle the
* assignment changes in its rebalance callback and then call *assign().
* 5. An incremental rebalancing is in progress.
* 6. A rebalance-induced rejoin is in progress.
*/
#define RD_KAFKA_CGRP_REBALANCING(rkcg) \
((RD_KAFKA_CGRP_HAS_JOINED(rkcg) && \
((rkcg)->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_WAIT_JOIN || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA)) || \
(rkcg)->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_WAIT_SYNC || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_TO_COMPLETE || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_INCR_UNASSIGN_TO_COMPLETE || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL || \
(rkcg)->rkcg_join_state == \
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL || \
(rkcg)->rkcg_rebalance_incr_assignment != NULL || \
(rkcg)->rkcg_rebalance_rejoin)
const char *rd_kafka_cgrp_state_names[] = {
"init", "term", "query-coord",
"wait-coord", "wait-broker", "wait-broker-transport",
"up"};
const char *rd_kafka_cgrp_join_state_names[] = {
"init",
"wait-join",
"wait-metadata",
"wait-sync",
"wait-assign-call",
"wait-unassign-call",
"wait-unassign-to-complete",
"wait-incr-unassign-to-complete",
"steady",
};
/**
* @brief Change the cgrp state.
*
* @returns 1 if the state was changed, else 0.
*/
static int rd_kafka_cgrp_set_state(rd_kafka_cgrp_t *rkcg, int state) {
if ((int)rkcg->rkcg_state == state)
return 0;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPSTATE",
"Group \"%.*s\" changed state %s -> %s "
"(join-state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_state_names[state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
rkcg->rkcg_state = state;
rkcg->rkcg_ts_statechange = rd_clock();
rd_kafka_brokers_broadcast_state_change(rkcg->rkcg_rk);
return 1;
}
void rd_kafka_cgrp_set_join_state(rd_kafka_cgrp_t *rkcg, int join_state) {
if ((int)rkcg->rkcg_join_state == join_state)
return;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPJOINSTATE",
"Group \"%.*s\" changed join state %s -> %s "
"(state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
rd_kafka_cgrp_join_state_names[join_state],
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
rkcg->rkcg_join_state = join_state;
}
void rd_kafka_cgrp_destroy_final(rd_kafka_cgrp_t *rkcg) {
rd_kafka_assert(rkcg->rkcg_rk, !rkcg->rkcg_subscription);
rd_kafka_assert(rkcg->rkcg_rk, !rkcg->rkcg_group_leader.members);
rd_kafka_cgrp_set_member_id(rkcg, NULL);
if (rkcg->rkcg_group_instance_id)
rd_kafkap_str_destroy(rkcg->rkcg_group_instance_id);
rd_kafka_q_destroy_owner(rkcg->rkcg_q);
rd_kafka_q_destroy_owner(rkcg->rkcg_ops);
rd_kafka_q_destroy_owner(rkcg->rkcg_wait_coord_q);
rd_kafka_assert(rkcg->rkcg_rk, TAILQ_EMPTY(&rkcg->rkcg_topics));
rd_kafka_assert(rkcg->rkcg_rk, rd_list_empty(&rkcg->rkcg_toppars));
rd_list_destroy(&rkcg->rkcg_toppars);
rd_list_destroy(rkcg->rkcg_subscribed_topics);
rd_kafka_topic_partition_list_destroy(rkcg->rkcg_errored_topics);
if (rkcg->rkcg_assignor && rkcg->rkcg_assignor->rkas_destroy_state_cb)
rkcg->rkcg_assignor->rkas_destroy_state_cb(
rkcg->rkcg_assignor_state);
rd_free(rkcg);
}
/**
* @brief Update the absolute session timeout following a successfull
* response from the coordinator.
* This timeout is used to enforce the session timeout in the
* consumer itself.
*
* @param reset if true the timeout is updated even if the session has expired.
*/
static RD_INLINE void
rd_kafka_cgrp_update_session_timeout(rd_kafka_cgrp_t *rkcg, rd_bool_t reset) {
if (reset || rkcg->rkcg_ts_session_timeout != 0)
rkcg->rkcg_ts_session_timeout =
rd_clock() +
(rkcg->rkcg_rk->rk_conf.group_session_timeout_ms * 1000);
}
rd_kafka_cgrp_t *rd_kafka_cgrp_new(rd_kafka_t *rk,
const rd_kafkap_str_t *group_id,
const rd_kafkap_str_t *client_id) {
rd_kafka_cgrp_t *rkcg;
rkcg = rd_calloc(1, sizeof(*rkcg));
rkcg->rkcg_rk = rk;
rkcg->rkcg_group_id = group_id;
rkcg->rkcg_client_id = client_id;
rkcg->rkcg_coord_id = -1;
rkcg->rkcg_generation_id = -1;
rkcg->rkcg_wait_resp = -1;
rkcg->rkcg_ops = rd_kafka_q_new(rk);
rkcg->rkcg_ops->rkq_serve = rd_kafka_cgrp_op_serve;
rkcg->rkcg_ops->rkq_opaque = rkcg;
rkcg->rkcg_wait_coord_q = rd_kafka_q_new(rk);
rkcg->rkcg_wait_coord_q->rkq_serve = rkcg->rkcg_ops->rkq_serve;
rkcg->rkcg_wait_coord_q->rkq_opaque = rkcg->rkcg_ops->rkq_opaque;
rkcg->rkcg_q = rd_kafka_q_new(rk);
rkcg->rkcg_group_instance_id =
rd_kafkap_str_new(rk->rk_conf.group_instance_id, -1);
TAILQ_INIT(&rkcg->rkcg_topics);
rd_list_init(&rkcg->rkcg_toppars, 32, NULL);
rd_kafka_cgrp_set_member_id(rkcg, "");
rkcg->rkcg_subscribed_topics =
rd_list_new(0, (void *)rd_kafka_topic_info_destroy);
rd_interval_init(&rkcg->rkcg_coord_query_intvl);
rd_interval_init(&rkcg->rkcg_heartbeat_intvl);
rd_interval_init(&rkcg->rkcg_join_intvl);
rd_interval_init(&rkcg->rkcg_timeout_scan_intvl);
rd_atomic32_init(&rkcg->rkcg_assignment_lost, rd_false);
rd_atomic32_init(&rkcg->rkcg_terminated, rd_false);
rkcg->rkcg_errored_topics = rd_kafka_topic_partition_list_new(0);
/* Create a logical group coordinator broker to provide
* a dedicated connection for group coordination.
* This is needed since JoinGroup may block for up to
* max.poll.interval.ms, effectively blocking and timing out
* any other protocol requests (such as Metadata).
* The address for this broker will be updated when
* the group coordinator is assigned. */
rkcg->rkcg_coord = rd_kafka_broker_add_logical(rk, "GroupCoordinator");
if (rk->rk_conf.enable_auto_commit &&
rk->rk_conf.auto_commit_interval_ms > 0)
rd_kafka_timer_start(
&rk->rk_timers, &rkcg->rkcg_offset_commit_tmr,
rk->rk_conf.auto_commit_interval_ms * 1000ll,
rd_kafka_cgrp_offset_commit_tmr_cb, rkcg);
return rkcg;
}
/**
* @brief Set the group coordinator broker.
*/
static void rd_kafka_cgrp_coord_set_broker(rd_kafka_cgrp_t *rkcg,
rd_kafka_broker_t *rkb) {
rd_assert(rkcg->rkcg_curr_coord == NULL);
rd_assert(RD_KAFKA_CGRP_BROKER_IS_COORD(rkcg, rkb));
rkcg->rkcg_curr_coord = rkb;
rd_kafka_broker_keep(rkb);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COORDSET",
"Group \"%.*s\" coordinator set to broker %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_broker_name(rkb));
/* Reset query interval to trigger an immediate
* coord query if required */
if (!rd_interval_disabled(&rkcg->rkcg_coord_query_intvl))
rd_interval_reset(&rkcg->rkcg_coord_query_intvl);
rd_kafka_cgrp_set_state(rkcg,
RD_KAFKA_CGRP_STATE_WAIT_BROKER_TRANSPORT);
rd_kafka_broker_persistent_connection_add(
rkcg->rkcg_coord, &rkcg->rkcg_coord->rkb_persistconn.coord);
/* Set the logical coordinator's nodename to the
* proper broker's nodename, this will trigger a (re)connect
* to the new address. */
rd_kafka_broker_set_nodename(rkcg->rkcg_coord, rkb);
}
/**
* @brief Reset/clear the group coordinator broker.
*/
static void rd_kafka_cgrp_coord_clear_broker(rd_kafka_cgrp_t *rkcg) {
rd_kafka_broker_t *rkb = rkcg->rkcg_curr_coord;
rd_assert(rkcg->rkcg_curr_coord);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COORDCLEAR",
"Group \"%.*s\" broker %s is no longer coordinator",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_broker_name(rkb));
rd_assert(rkcg->rkcg_coord);
rd_kafka_broker_persistent_connection_del(
rkcg->rkcg_coord, &rkcg->rkcg_coord->rkb_persistconn.coord);
/* Clear the ephemeral broker's nodename.
* This will also trigger a disconnect. */
rd_kafka_broker_set_nodename(rkcg->rkcg_coord, NULL);
rkcg->rkcg_curr_coord = NULL;
rd_kafka_broker_destroy(rkb); /* from set_coord_broker() */
}
/**
* @brief Update/set the group coordinator.
*
* Will do nothing if there's been no change.
*
* @returns 1 if the coordinator, or state, was updated, else 0.
*/
static int rd_kafka_cgrp_coord_update(rd_kafka_cgrp_t *rkcg, int32_t coord_id) {
/* Don't do anything while terminating */
if (rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_TERM)
return 0;
/* Check if coordinator changed */
if (rkcg->rkcg_coord_id != coord_id) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPCOORD",
"Group \"%.*s\" changing coordinator %" PRId32
" -> %" PRId32,
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rkcg->rkcg_coord_id, coord_id);
/* Update coord id */
rkcg->rkcg_coord_id = coord_id;
/* Clear previous broker handle, if any */
if (rkcg->rkcg_curr_coord)
rd_kafka_cgrp_coord_clear_broker(rkcg);
}
if (rkcg->rkcg_curr_coord) {
/* There is already a known coordinator and a
* corresponding broker handle. */
if (rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP)
return rd_kafka_cgrp_set_state(
rkcg, RD_KAFKA_CGRP_STATE_WAIT_BROKER_TRANSPORT);
} else if (rkcg->rkcg_coord_id != -1) {
rd_kafka_broker_t *rkb;
/* Try to find the coordinator broker handle */
rd_kafka_rdlock(rkcg->rkcg_rk);
rkb = rd_kafka_broker_find_by_nodeid(rkcg->rkcg_rk, coord_id);
rd_kafka_rdunlock(rkcg->rkcg_rk);
/* It is possible, due to stale metadata, that the
* coordinator id points to a broker we still don't know
* about. In this case the client will continue
* querying metadata and querying for the coordinator
* until a match is found. */
if (rkb) {
/* Coordinator is known and broker handle exists */
rd_kafka_cgrp_coord_set_broker(rkcg, rkb);
rd_kafka_broker_destroy(rkb); /*from find_by_nodeid()*/
return 1;
} else {
/* Coordinator is known but no corresponding
* broker handle. */
return rd_kafka_cgrp_set_state(
rkcg, RD_KAFKA_CGRP_STATE_WAIT_BROKER);
}
} else {
/* Coordinator still not known, re-query */
if (rkcg->rkcg_state >= RD_KAFKA_CGRP_STATE_WAIT_COORD)
return rd_kafka_cgrp_set_state(
rkcg, RD_KAFKA_CGRP_STATE_QUERY_COORD);
}
return 0; /* no change */
}
/**
* Handle FindCoordinator response
*/
static void rd_kafka_cgrp_handle_FindCoordinator(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
void *opaque) {
const int log_decode_errors = LOG_ERR;
int16_t ErrorCode = 0;
int32_t CoordId;
rd_kafkap_str_t CoordHost = RD_ZERO_INIT;
int32_t CoordPort;
rd_kafka_cgrp_t *rkcg = opaque;
struct rd_kafka_metadata_broker mdb = RD_ZERO_INIT;
char *errstr = NULL;
int actions;
if (likely(!(ErrorCode = err))) {
if (rkbuf->rkbuf_reqhdr.ApiVersion >= 1)
rd_kafka_buf_read_throttle_time(rkbuf);
rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
if (rkbuf->rkbuf_reqhdr.ApiVersion >= 1) {
rd_kafkap_str_t ErrorMsg;
rd_kafka_buf_read_str(rkbuf, &ErrorMsg);
if (!RD_KAFKAP_STR_IS_NULL(&ErrorMsg))
RD_KAFKAP_STR_DUPA(&errstr, &ErrorMsg);
}
rd_kafka_buf_read_i32(rkbuf, &CoordId);
rd_kafka_buf_read_str(rkbuf, &CoordHost);
rd_kafka_buf_read_i32(rkbuf, &CoordPort);
}
if (ErrorCode)
goto err;
mdb.id = CoordId;
RD_KAFKAP_STR_DUPA(&mdb.host, &CoordHost);
mdb.port = CoordPort;
rd_rkb_dbg(rkb, CGRP, "CGRPCOORD",
"Group \"%.*s\" coordinator is %s:%i id %" PRId32,
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), mdb.host, mdb.port,
mdb.id);
rd_kafka_broker_update(rkb->rkb_rk, rkb->rkb_proto, &mdb, NULL);
rd_kafka_cgrp_coord_update(rkcg, CoordId);
rd_kafka_cgrp_serve(rkcg); /* Serve updated state, if possible */
return;
err_parse: /* Parse error */
ErrorCode = rkbuf->rkbuf_err;
/* FALLTHRU */
err:
if (!errstr)
errstr = (char *)rd_kafka_err2str(ErrorCode);
rd_rkb_dbg(rkb, CGRP, "CGRPCOORD",
"Group \"%.*s\" FindCoordinator response error: %s: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_err2name(ErrorCode), errstr);
if (ErrorCode == RD_KAFKA_RESP_ERR__DESTROY)
return;
actions = rd_kafka_err_action(
rkb, ErrorCode, request,
RD_KAFKA_ERR_ACTION_RETRY | RD_KAFKA_ERR_ACTION_REFRESH,
RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE,
RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR__TRANSPORT,
RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR__TIMED_OUT,
RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE,
RD_KAFKA_ERR_ACTION_END);
if (actions & RD_KAFKA_ERR_ACTION_REFRESH) {
rd_kafka_cgrp_coord_update(rkcg, -1);
} else {
if (!(actions & RD_KAFKA_ERR_ACTION_RETRY) &&
rkcg->rkcg_last_err != ErrorCode) {
/* Propagate non-retriable errors to the application */
rd_kafka_consumer_err(
rkcg->rkcg_q, rd_kafka_broker_id(rkb), ErrorCode, 0,
NULL, NULL, RD_KAFKA_OFFSET_INVALID,
"FindCoordinator response error: %s", errstr);
/* Suppress repeated errors */
rkcg->rkcg_last_err = ErrorCode;
}
/* Retries are performed by the timer-intervalled
* coord queries, continue querying */
rd_kafka_cgrp_set_state(rkcg, RD_KAFKA_CGRP_STATE_QUERY_COORD);
}
rd_kafka_cgrp_serve(rkcg); /* Serve updated state, if possible */
}
/**
* Query for coordinator.
* Ask any broker in state UP
*
* Locality: main thread
*/
void rd_kafka_cgrp_coord_query(rd_kafka_cgrp_t *rkcg, const char *reason) {
rd_kafka_broker_t *rkb;
rd_kafka_resp_err_t err;
rkb = rd_kafka_broker_any_usable(
rkcg->rkcg_rk, RD_POLL_NOWAIT, RD_DO_LOCK,
RD_KAFKA_FEATURE_BROKER_GROUP_COORD, "coordinator query");
if (!rkb) {
/* Reset the interval because there were no brokers. When a
* broker becomes available, we want to query it immediately. */
rd_interval_reset(&rkcg->rkcg_coord_query_intvl);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPQUERY",
"Group \"%.*s\": "
"no broker available for coordinator query: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), reason);
return;
}
rd_rkb_dbg(rkb, CGRP, "CGRPQUERY",
"Group \"%.*s\": querying for coordinator: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), reason);
err = rd_kafka_FindCoordinatorRequest(
rkb, RD_KAFKA_COORD_GROUP, rkcg->rkcg_group_id->str,
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_FindCoordinator, rkcg);
if (err) {
rd_rkb_dbg(rkb, CGRP, "CGRPQUERY",
"Group \"%.*s\": "
"unable to send coordinator query: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_err2str(err));
rd_kafka_broker_destroy(rkb);
return;
}
if (rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_QUERY_COORD)
rd_kafka_cgrp_set_state(rkcg, RD_KAFKA_CGRP_STATE_WAIT_COORD);
rd_kafka_broker_destroy(rkb);
/* Back off the next intervalled query since we just sent one. */
rd_interval_reset_to_now(&rkcg->rkcg_coord_query_intvl, 0);
}
/**
* @brief Mark the current coordinator as dead.
*
* @locality main thread
*/
void rd_kafka_cgrp_coord_dead(rd_kafka_cgrp_t *rkcg,
rd_kafka_resp_err_t err,
const char *reason) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COORD",
"Group \"%.*s\": "
"marking the coordinator (%" PRId32 ") dead: %s: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), rkcg->rkcg_coord_id,
rd_kafka_err2str(err), reason);
rd_kafka_cgrp_coord_update(rkcg, -1);
/* Re-query for coordinator */
rd_kafka_cgrp_set_state(rkcg, RD_KAFKA_CGRP_STATE_QUERY_COORD);
rd_kafka_cgrp_coord_query(rkcg, reason);
}
/**
* @returns a new reference to the current coordinator, if available, else NULL.
*
* @locality rdkafka main thread
* @locks_required none
* @locks_acquired none
*/
rd_kafka_broker_t *rd_kafka_cgrp_get_coord(rd_kafka_cgrp_t *rkcg) {
if (rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP || !rkcg->rkcg_coord)
return NULL;
rd_kafka_broker_keep(rkcg->rkcg_coord);
return rkcg->rkcg_coord;
}
/**
* @brief cgrp handling of LeaveGroup responses
* @param opaque must be the cgrp handle.
* @locality rdkafka main thread (unless err==ERR__DESTROY)
*/
static void rd_kafka_cgrp_handle_LeaveGroup(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
void *opaque) {
rd_kafka_cgrp_t *rkcg = opaque;
const int log_decode_errors = LOG_ERR;
int16_t ErrorCode = 0;
if (err) {
ErrorCode = err;
goto err;
}
if (request->rkbuf_reqhdr.ApiVersion >= 1)
rd_kafka_buf_read_throttle_time(rkbuf);
rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
err:
if (ErrorCode)
rd_kafka_dbg(rkb->rkb_rk, CGRP, "LEAVEGROUP",
"LeaveGroup response error in state %s: %s",
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_err2str(ErrorCode));
else
rd_kafka_dbg(rkb->rkb_rk, CGRP, "LEAVEGROUP",
"LeaveGroup response received in state %s",
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
if (ErrorCode != RD_KAFKA_RESP_ERR__DESTROY) {
rd_assert(thrd_is_current(rk->rk_thread));
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_WAIT_LEAVE;
rd_kafka_cgrp_try_terminate(rkcg);
}
return;
err_parse:
ErrorCode = rkbuf->rkbuf_err;
goto err;
}
static void rd_kafka_cgrp_leave(rd_kafka_cgrp_t *rkcg) {
char *member_id;
RD_KAFKAP_STR_DUPA(&member_id, rkcg->rkcg_member_id);
/* Leaving the group invalidates the member id, reset it
* now to avoid an ERR_UNKNOWN_MEMBER_ID on the next join. */
rd_kafka_cgrp_set_member_id(rkcg, "");
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WAIT_LEAVE) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "LEAVE",
"Group \"%.*s\": leave (in state %s): "
"LeaveGroupRequest already in-transit",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
return;
}
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "LEAVE",
"Group \"%.*s\": leave (in state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_WAIT_LEAVE;
if (rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_UP) {
rd_rkb_dbg(rkcg->rkcg_curr_coord, CONSUMER, "LEAVE",
"Leaving group");
rd_kafka_LeaveGroupRequest(
rkcg->rkcg_coord, rkcg->rkcg_group_id->str, member_id,
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_LeaveGroup, rkcg);
} else
rd_kafka_cgrp_handle_LeaveGroup(rkcg->rkcg_rk, rkcg->rkcg_coord,
RD_KAFKA_RESP_ERR__WAIT_COORD,
NULL, NULL, rkcg);
}
/**
* @brief Leave group, if desired.
*
* @returns true if a LeaveGroup was issued, else false.
*/
static rd_bool_t rd_kafka_cgrp_leave_maybe(rd_kafka_cgrp_t *rkcg) {
/* We were not instructed to leave in the first place. */
if (!(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE))
return rd_false;
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE;
/* Don't send Leave when termating with NO_CONSUMER_CLOSE flag */
if (rd_kafka_destroy_flags_no_consumer_close(rkcg->rkcg_rk))
return rd_false;
/* KIP-345: Static group members must not send a LeaveGroupRequest
* on termination. */
if (RD_KAFKA_CGRP_IS_STATIC_MEMBER(rkcg) &&
rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE)
return rd_false;
rd_kafka_cgrp_leave(rkcg);
return rd_true;
}
/**
* @brief Enqueues a rebalance op, delegating responsibility of calling
* incremental_assign / incremental_unassign to the application.
* If there is no rebalance handler configured, or the action
* should not be delegated to the application for some other
* reason, incremental_assign / incremental_unassign will be called
* automatically, immediately.
*
* @param rejoin whether or not to rejoin the group following completion
* of the incremental assign / unassign.
*
* @remarks does not take ownership of \p partitions.
*/
void rd_kafka_rebalance_op_incr(rd_kafka_cgrp_t *rkcg,
rd_kafka_resp_err_t err,
rd_kafka_topic_partition_list_t *partitions,
rd_bool_t rejoin,
const char *reason) {
rd_kafka_error_t *error;
/* Flag to rejoin after completion of the incr_assign or incr_unassign,
if required. */
rkcg->rkcg_rebalance_rejoin = rejoin;
rd_kafka_wrlock(rkcg->rkcg_rk);
rkcg->rkcg_c.ts_rebalance = rd_clock();
rkcg->rkcg_c.rebalance_cnt++;
rd_kafka_wrunlock(rkcg->rkcg_rk);
if (rd_kafka_destroy_flags_no_consumer_close(rkcg->rkcg_rk) ||
rd_kafka_fatal_error_code(rkcg->rkcg_rk)) {
/* Total unconditional unassign in these cases */
rd_kafka_cgrp_unassign(rkcg);
/* Now serve the assignment to make updates */
rd_kafka_assignment_serve(rkcg->rkcg_rk);
goto done;
}
rd_kafka_cgrp_set_join_state(
rkcg, err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS
? RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL
: RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL);
/* Schedule application rebalance callback/event if enabled */
if (rkcg->rkcg_rk->rk_conf.enabled_events & RD_KAFKA_EVENT_REBALANCE) {
rd_kafka_op_t *rko;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "ASSIGN",
"Group \"%s\": delegating incremental %s of %d "
"partition(s) to application on queue %s: %s",
rkcg->rkcg_group_id->str,
err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS
? "revoke"
: "assign",
partitions->cnt,
rd_kafka_q_dest_name(rkcg->rkcg_q), reason);
/* Pause currently assigned partitions while waiting for
* rebalance callback to get called to make sure the
* application will not receive any more messages that
* might block it from serving the rebalance callback
* and to not process messages for partitions it
* might have lost in the rebalance. */
rd_kafka_assignment_pause(rkcg->rkcg_rk,
"incremental rebalance");
rko = rd_kafka_op_new(RD_KAFKA_OP_REBALANCE);
rko->rko_err = err;
rko->rko_u.rebalance.partitions =
rd_kafka_topic_partition_list_copy(partitions);
if (rd_kafka_q_enq(rkcg->rkcg_q, rko))
goto done; /* Rebalance op successfully enqueued */
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRP",
"Group \"%s\": ops queue is disabled, not "
"delegating partition %s to application",
rkcg->rkcg_group_id->str,
err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS
? "unassign"
: "assign");
/* FALLTHRU */
}
/* No application rebalance callback/event handler, or it is not
* available, do the assign/unassign ourselves.
* We need to be careful here not to trigger assignment_serve()
* since it may call into the cgrp code again, in which case we
* can't really track what the outcome state will be. */
if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS)
error = rd_kafka_cgrp_incremental_assign(rkcg, partitions);
else
error = rd_kafka_cgrp_incremental_unassign(rkcg, partitions);
if (error) {
rd_kafka_log(rkcg->rkcg_rk, LOG_ERR, "REBALANCE",
"Group \"%s\": internal incremental %s "
"of %d partition(s) failed: %s: "
"unassigning all partitions and rejoining",
rkcg->rkcg_group_id->str,
err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS
? "unassign"
: "assign",
partitions->cnt, rd_kafka_error_string(error));
rd_kafka_error_destroy(error);
rd_kafka_cgrp_set_join_state(rkcg,
/* This is a clean state for
* assignment_done() to rejoin
* from. */
RD_KAFKA_CGRP_JOIN_STATE_STEADY);
rd_kafka_assignment_clear(rkcg->rkcg_rk);
}
/* Now serve the assignment to make updates */
rd_kafka_assignment_serve(rkcg->rkcg_rk);
done:
/* Update the current group assignment based on the
* added/removed partitions. */
rd_kafka_cgrp_group_assignment_modify(
rkcg, err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS, partitions);
}
/**
* @brief Enqueues a rebalance op, delegating responsibility of calling
* assign / unassign to the application. If there is no rebalance
* handler configured, or the action should not be delegated to the
* application for some other reason, assign / unassign will be
* called automatically.
*
* @remarks \p partitions is copied.
*/
static void rd_kafka_rebalance_op(rd_kafka_cgrp_t *rkcg,
rd_kafka_resp_err_t err,
rd_kafka_topic_partition_list_t *assignment,
const char *reason) {
rd_kafka_error_t *error;
rd_kafka_wrlock(rkcg->rkcg_rk);
rkcg->rkcg_c.ts_rebalance = rd_clock();
rkcg->rkcg_c.rebalance_cnt++;
rd_kafka_wrunlock(rkcg->rkcg_rk);
if (rd_kafka_destroy_flags_no_consumer_close(rkcg->rkcg_rk) ||
rd_kafka_fatal_error_code(rkcg->rkcg_rk)) {
/* Unassign */
rd_kafka_cgrp_unassign(rkcg);
/* Now serve the assignment to make updates */
rd_kafka_assignment_serve(rkcg->rkcg_rk);
goto done;
}
rd_assert(assignment != NULL);
rd_kafka_cgrp_set_join_state(
rkcg, err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS
? RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL
: RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL);
/* Schedule application rebalance callback/event if enabled */
if (rkcg->rkcg_rk->rk_conf.enabled_events & RD_KAFKA_EVENT_REBALANCE) {
rd_kafka_op_t *rko;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "ASSIGN",
"Group \"%s\": delegating %s of %d partition(s) "
"to application on queue %s: %s",
rkcg->rkcg_group_id->str,
err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS
? "revoke"
: "assign",
assignment->cnt,
rd_kafka_q_dest_name(rkcg->rkcg_q), reason);
/* Pause currently assigned partitions while waiting for
* rebalance callback to get called to make sure the
* application will not receive any more messages that
* might block it from serving the rebalance callback
* and to not process messages for partitions it
* might have lost in the rebalance. */
rd_kafka_assignment_pause(rkcg->rkcg_rk, "rebalance");
rko = rd_kafka_op_new(RD_KAFKA_OP_REBALANCE);
rko->rko_err = err;
rko->rko_u.rebalance.partitions =
rd_kafka_topic_partition_list_copy(assignment);
if (rd_kafka_q_enq(rkcg->rkcg_q, rko))
goto done; /* Rebalance op successfully enqueued */
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRP",
"Group \"%s\": ops queue is disabled, not "
"delegating partition %s to application",
rkcg->rkcg_group_id->str,
err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS
? "unassign"
: "assign");
/* FALLTHRU */
}
/* No application rebalance callback/event handler, or it is not
* available, do the assign/unassign ourselves.
* We need to be careful here not to trigger assignment_serve()
* since it may call into the cgrp code again, in which case we
* can't really track what the outcome state will be. */
if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS)
error = rd_kafka_cgrp_assign(rkcg, assignment);
else
error = rd_kafka_cgrp_unassign(rkcg);
if (error) {
rd_kafka_log(rkcg->rkcg_rk, LOG_ERR, "REBALANCE",
"Group \"%s\": internal %s "
"of %d partition(s) failed: %s: "
"unassigning all partitions and rejoining",
rkcg->rkcg_group_id->str,
err == RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS
? "unassign"
: "assign",
rkcg->rkcg_group_assignment->cnt,
rd_kafka_error_string(error));
rd_kafka_error_destroy(error);
rd_kafka_cgrp_set_join_state(rkcg,
/* This is a clean state for
* assignment_done() to rejoin
* from. */
RD_KAFKA_CGRP_JOIN_STATE_STEADY);
rd_kafka_assignment_clear(rkcg->rkcg_rk);
}
/* Now serve the assignment to make updates */
rd_kafka_assignment_serve(rkcg->rkcg_rk);
done:
if (err == RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS)
rd_kafka_cgrp_group_assignment_set(rkcg, assignment);
else
rd_kafka_cgrp_group_assignment_set(rkcg, NULL);
}
/**
* @brief Rejoin the group.
*
* @remark This function must not have any side-effects but setting the
* join state.
*/
static void rd_kafka_cgrp_rejoin(rd_kafka_cgrp_t *rkcg, const char *fmt, ...)
RD_FORMAT(printf, 2, 3);
static void rd_kafka_cgrp_rejoin(rd_kafka_cgrp_t *rkcg, const char *fmt, ...) {
char reason[512];
va_list ap;
char astr[128];
va_start(ap, fmt);
rd_vsnprintf(reason, sizeof(reason), fmt, ap);
va_end(ap);
if (rkcg->rkcg_group_assignment)
rd_snprintf(astr, sizeof(astr), " with %d owned partition(s)",
rkcg->rkcg_group_assignment->cnt);
else
rd_snprintf(astr, sizeof(astr), " without an assignment");
if (rkcg->rkcg_subscription || rkcg->rkcg_next_subscription) {
rd_kafka_dbg(
rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "REJOIN",
"Group \"%s\": %s group%s: %s", rkcg->rkcg_group_id->str,
rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT
? "Joining"
: "Rejoining",
astr, reason);
} else {
rd_kafka_dbg(
rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "NOREJOIN",
"Group \"%s\": Not %s group%s: %s: "
"no subscribed topics",
rkcg->rkcg_group_id->str,
rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT
? "joining"
: "rejoining",
astr, reason);
rd_kafka_cgrp_leave_maybe(rkcg);
}
rd_kafka_cgrp_set_join_state(rkcg, RD_KAFKA_CGRP_JOIN_STATE_INIT);
}
/**
* @brief Collect all assigned or owned partitions from group members.
* The member field of each result element is set to the associated
* group member. The members_match field is set to rd_false.
*
* @param members Array of group members.
* @param member_cnt Number of elements in members.
* @param par_cnt The total number of partitions expected to be collected.
* @param collect_owned If rd_true, rkgm_owned partitions will be collected,
* else rkgm_assignment partitions will be collected.
*/
static map_toppar_member_info_t *
rd_kafka_collect_partitions(const rd_kafka_group_member_t *members,
size_t member_cnt,
size_t par_cnt,
rd_bool_t collect_owned) {
size_t i;
map_toppar_member_info_t *collected = rd_calloc(1, sizeof(*collected));
RD_MAP_INIT(collected, par_cnt, rd_kafka_topic_partition_cmp,
rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free,
PartitionMemberInfo_free);
for (i = 0; i < member_cnt; i++) {
size_t j;
const rd_kafka_group_member_t *rkgm = &members[i];
const rd_kafka_topic_partition_list_t *toppars =
collect_owned ? rkgm->rkgm_owned : rkgm->rkgm_assignment;
for (j = 0; j < (size_t)toppars->cnt; j++) {
rd_kafka_topic_partition_t *rktpar =
rd_kafka_topic_partition_copy(&toppars->elems[j]);
PartitionMemberInfo_t *pmi =
PartitionMemberInfo_new(rkgm, rd_false);
RD_MAP_SET(collected, rktpar, pmi);
}
}
return collected;
}
/**
* @brief Set intersection. Returns a set of all elements of \p a that
* are also elements of \p b. Additionally, compares the members
* field of matching elements from \p a and \p b and if not NULL
* and equal, sets the members_match field in the result element
* to rd_true and the member field to equal that of the elements,
* else sets the members_match field to rd_false and member field
* to NULL.
*/
static map_toppar_member_info_t *
rd_kafka_member_partitions_intersect(map_toppar_member_info_t *a,
map_toppar_member_info_t *b) {
const rd_kafka_topic_partition_t *key;
const PartitionMemberInfo_t *a_v;
map_toppar_member_info_t *intersection =
rd_calloc(1, sizeof(*intersection));
RD_MAP_INIT(
intersection, RD_MIN(a ? RD_MAP_CNT(a) : 1, b ? RD_MAP_CNT(b) : 1),
rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free, PartitionMemberInfo_free);
if (!a || !b)
return intersection;
RD_MAP_FOREACH(key, a_v, a) {
rd_bool_t members_match;
const PartitionMemberInfo_t *b_v = RD_MAP_GET(b, key);
if (b_v == NULL)
continue;
members_match =
a_v->member && b_v->member &&
rd_kafka_group_member_cmp(a_v->member, b_v->member) == 0;
RD_MAP_SET(intersection, rd_kafka_topic_partition_copy(key),
PartitionMemberInfo_new(b_v->member, members_match));
}
return intersection;
}
/**
* @brief Set subtraction. Returns a set of all elements of \p a
* that are not elements of \p b. Sets the member field in
* elements in the returned set to equal that of the
* corresponding element in \p a
*/
static map_toppar_member_info_t *
rd_kafka_member_partitions_subtract(map_toppar_member_info_t *a,
map_toppar_member_info_t *b) {
const rd_kafka_topic_partition_t *key;
const PartitionMemberInfo_t *a_v;
map_toppar_member_info_t *difference =
rd_calloc(1, sizeof(*difference));
RD_MAP_INIT(difference, a ? RD_MAP_CNT(a) : 1,
rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free,
PartitionMemberInfo_free);
if (!a)
return difference;
RD_MAP_FOREACH(key, a_v, a) {
const PartitionMemberInfo_t *b_v =
b ? RD_MAP_GET(b, key) : NULL;
if (!b_v)
RD_MAP_SET(
difference, rd_kafka_topic_partition_copy(key),
PartitionMemberInfo_new(a_v->member, rd_false));
}
return difference;
}
/**
* @brief Adjust the partition assignment as provided by the assignor
* according to the COOPERATIVE protocol.
*/
static void rd_kafka_cooperative_protocol_adjust_assignment(
rd_kafka_cgrp_t *rkcg,
rd_kafka_group_member_t *members,
int member_cnt) {
/* https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafk\
a+Consumer+Incremental+Rebalance+Protocol */
int i;
int expected_max_assignment_size;
int total_assigned = 0;
int not_revoking = 0;
size_t par_cnt = 0;
const rd_kafka_topic_partition_t *toppar;
const PartitionMemberInfo_t *pmi;
map_toppar_member_info_t *assigned;
map_toppar_member_info_t *owned;
map_toppar_member_info_t *maybe_revoking;
map_toppar_member_info_t *ready_to_migrate;
map_toppar_member_info_t *unknown_but_owned;
for (i = 0; i < member_cnt; i++)
par_cnt += members[i].rkgm_owned->cnt;
assigned = rd_kafka_collect_partitions(members, member_cnt, par_cnt,
rd_false /*assigned*/);
owned = rd_kafka_collect_partitions(members, member_cnt, par_cnt,
rd_true /*owned*/);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRP",
"Group \"%s\": Partitions owned by members: %d, "
"partitions assigned by assignor: %d",
rkcg->rkcg_group_id->str, (int)RD_MAP_CNT(owned),
(int)RD_MAP_CNT(assigned));
/* Still owned by some members */
maybe_revoking = rd_kafka_member_partitions_intersect(assigned, owned);
/* Not previously owned by anyone */
ready_to_migrate = rd_kafka_member_partitions_subtract(assigned, owned);
/* Don't exist in assigned partitions */
unknown_but_owned =
rd_kafka_member_partitions_subtract(owned, assigned);
/* Rough guess at a size that is a bit higher than
* the maximum number of partitions likely to be
* assigned to any partition. */
expected_max_assignment_size =
(int)(RD_MAP_CNT(assigned) / member_cnt) + 4;
for (i = 0; i < member_cnt; i++) {
rd_kafka_group_member_t *rkgm = &members[i];
rd_kafka_topic_partition_list_destroy(rkgm->rkgm_assignment);
rkgm->rkgm_assignment = rd_kafka_topic_partition_list_new(
expected_max_assignment_size);
}
/* For maybe-revoking-partitions, check if the owner has
* changed. If yes, exclude them from the assigned-partitions
* list to the new owner. The old owner will realize it does
* not own it any more, revoke it and then trigger another
* rebalance for these partitions to finally be reassigned.
*/
RD_MAP_FOREACH(toppar, pmi, maybe_revoking) {
if (!pmi->members_match)
/* Owner has changed. */
continue;
/* Owner hasn't changed. */
rd_kafka_topic_partition_list_add(pmi->member->rkgm_assignment,
toppar->topic,
toppar->partition);
total_assigned++;
not_revoking++;
}
/* For ready-to-migrate-partitions, it is safe to move them
* to the new member immediately since we know no one owns
* it before, and hence we can encode the owner from the
* newly-assigned-partitions directly.
*/
RD_MAP_FOREACH(toppar, pmi, ready_to_migrate) {
rd_kafka_topic_partition_list_add(pmi->member->rkgm_assignment,
toppar->topic,
toppar->partition);
total_assigned++;
}
/* For unknown-but-owned-partitions, it is also safe to just
* give them back to whoever claimed to be their owners by
* encoding them directly as well. If this is due to a topic
* metadata update, then a later rebalance will be triggered
* anyway.
*/
RD_MAP_FOREACH(toppar, pmi, unknown_but_owned) {
rd_kafka_topic_partition_list_add(pmi->member->rkgm_assignment,
toppar->topic,
toppar->partition);
total_assigned++;
}
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRP",
"Group \"%s\": COOPERATIVE protocol collection sizes: "
"maybe revoking: %d, ready to migrate: %d, unknown but "
"owned: %d",
rkcg->rkcg_group_id->str, (int)RD_MAP_CNT(maybe_revoking),
(int)RD_MAP_CNT(ready_to_migrate),
(int)RD_MAP_CNT(unknown_but_owned));
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRP",
"Group \"%s\": %d partitions assigned to consumers",
rkcg->rkcg_group_id->str, total_assigned);
RD_MAP_DESTROY_AND_FREE(maybe_revoking);
RD_MAP_DESTROY_AND_FREE(ready_to_migrate);
RD_MAP_DESTROY_AND_FREE(unknown_but_owned);
RD_MAP_DESTROY_AND_FREE(assigned);
RD_MAP_DESTROY_AND_FREE(owned);
}
/**
* @brief Parses and handles the MemberState from a SyncGroupResponse.
*/
static void rd_kafka_cgrp_handle_SyncGroup_memberstate(
rd_kafka_cgrp_t *rkcg,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
const rd_kafkap_bytes_t *member_state) {
rd_kafka_buf_t *rkbuf = NULL;
rd_kafka_topic_partition_list_t *assignment = NULL;
const int log_decode_errors = LOG_ERR;
int16_t Version;
rd_kafkap_bytes_t UserData;
/* Dont handle new assignments when terminating */
if (!err && rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE)
err = RD_KAFKA_RESP_ERR__DESTROY;
if (err)
goto err;
if (RD_KAFKAP_BYTES_LEN(member_state) == 0) {
/* Empty assignment. */
assignment = rd_kafka_topic_partition_list_new(0);
memset(&UserData, 0, sizeof(UserData));
goto done;
}
/* Parse assignment from MemberState */
rkbuf = rd_kafka_buf_new_shadow(
member_state->data, RD_KAFKAP_BYTES_LEN(member_state), NULL);
/* Protocol parser needs a broker handle to log errors on. */
if (rkb) {
rkbuf->rkbuf_rkb = rkb;
rd_kafka_broker_keep(rkb);
} else
rkbuf->rkbuf_rkb = rd_kafka_broker_internal(rkcg->rkcg_rk);
rd_kafka_buf_read_i16(rkbuf, &Version);
const rd_kafka_topic_partition_field_t fields[] = {
RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
RD_KAFKA_TOPIC_PARTITION_FIELD_END};
if (!(assignment =
rd_kafka_buf_read_topic_partitions(rkbuf, 0, fields)))
goto err_parse;
rd_kafka_buf_read_bytes(rkbuf, &UserData);
done:
rd_kafka_cgrp_update_session_timeout(rkcg, rd_true /*reset timeout*/);
rd_assert(rkcg->rkcg_assignor);
if (rkcg->rkcg_assignor->rkas_on_assignment_cb) {
char *member_id;
RD_KAFKAP_STR_DUPA(&member_id, rkcg->rkcg_member_id);
rd_kafka_consumer_group_metadata_t *cgmd =
rd_kafka_consumer_group_metadata_new_with_genid(
rkcg->rkcg_rk->rk_conf.group_id_str,
rkcg->rkcg_generation_id, member_id,
rkcg->rkcg_rk->rk_conf.group_instance_id);
rkcg->rkcg_assignor->rkas_on_assignment_cb(
rkcg->rkcg_assignor, &(rkcg->rkcg_assignor_state),
assignment, &UserData, cgmd);
rd_kafka_consumer_group_metadata_destroy(cgmd);
}
// FIXME: Remove when we're done debugging.
rd_kafka_topic_partition_list_log(rkcg->rkcg_rk, "ASSIGNMENT",
RD_KAFKA_DBG_CGRP, assignment);
/* Set the new assignment */
rd_kafka_cgrp_handle_assignment(rkcg, assignment);
rd_kafka_topic_partition_list_destroy(assignment);
if (rkbuf)
rd_kafka_buf_destroy(rkbuf);
return;
err_parse:
err = rkbuf->rkbuf_err;
err:
if (rkbuf)
rd_kafka_buf_destroy(rkbuf);
if (assignment)
rd_kafka_topic_partition_list_destroy(assignment);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "GRPSYNC",
"Group \"%s\": synchronization failed: %s: rejoining",
rkcg->rkcg_group_id->str, rd_kafka_err2str(err));
if (err == RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID)
rd_kafka_set_fatal_error(rkcg->rkcg_rk, err,
"Fatal consumer error: %s",
rd_kafka_err2str(err));
else if (err == RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION)
rkcg->rkcg_generation_id = -1;
else if (err == RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID)
rd_kafka_cgrp_set_member_id(rkcg, "");
if (rd_kafka_cgrp_rebalance_protocol(rkcg) ==
RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE &&
(err == RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION ||
err == RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID))
rd_kafka_cgrp_revoke_all_rejoin(
rkcg, rd_true /*assignment is lost*/,
rd_true /*this consumer is initiating*/, "SyncGroup error");
else
rd_kafka_cgrp_rejoin(rkcg, "SyncGroup error: %s",
rd_kafka_err2str(err));
}
/**
* @brief Cgrp handler for SyncGroup responses. opaque must be the cgrp handle.
*/
static void rd_kafka_cgrp_handle_SyncGroup(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
void *opaque) {
rd_kafka_cgrp_t *rkcg = opaque;
const int log_decode_errors = LOG_ERR;
int16_t ErrorCode = 0;
rd_kafkap_bytes_t MemberState = RD_ZERO_INIT;
int actions;
if (rkcg->rkcg_join_state != RD_KAFKA_CGRP_JOIN_STATE_WAIT_SYNC) {
rd_kafka_dbg(
rkb->rkb_rk, CGRP, "SYNCGROUP",
"SyncGroup response: discarding outdated request "
"(now in join-state %s)",
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
rd_kafka_cgrp_clear_wait_resp(rkcg, RD_KAFKAP_SyncGroup);
return;
}
if (err) {
ErrorCode = err;
goto err;
}
if (request->rkbuf_reqhdr.ApiVersion >= 1)
rd_kafka_buf_read_throttle_time(rkbuf);
rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
rd_kafka_buf_read_bytes(rkbuf, &MemberState);
err:
actions = rd_kafka_err_action(rkb, ErrorCode, request,
RD_KAFKA_ERR_ACTION_END);
if (actions & RD_KAFKA_ERR_ACTION_REFRESH) {
/* Re-query for coordinator */
rd_kafka_cgrp_op(rkcg, NULL, RD_KAFKA_NO_REPLYQ,
RD_KAFKA_OP_COORD_QUERY, ErrorCode);
/* FALLTHRU */
}
if (actions & RD_KAFKA_ERR_ACTION_RETRY) {
if (rd_kafka_buf_retry(rkb, request))
return;
/* FALLTHRU */
}
rd_kafka_dbg(rkb->rkb_rk, CGRP, "SYNCGROUP",
"SyncGroup response: %s (%d bytes of MemberState data)",
rd_kafka_err2str(ErrorCode),
RD_KAFKAP_BYTES_LEN(&MemberState));
rd_kafka_cgrp_clear_wait_resp(rkcg, RD_KAFKAP_SyncGroup);
if (ErrorCode == RD_KAFKA_RESP_ERR__DESTROY)
return; /* Termination */
rd_kafka_cgrp_handle_SyncGroup_memberstate(rkcg, rkb, ErrorCode,
&MemberState);
return;
err_parse:
ErrorCode = rkbuf->rkbuf_err;
goto err;
}
/**
* @brief Run group assignment.
*/
static void rd_kafka_cgrp_assignor_run(rd_kafka_cgrp_t *rkcg,
rd_kafka_assignor_t *rkas,
rd_kafka_resp_err_t err,
rd_kafka_metadata_t *metadata,
rd_kafka_group_member_t *members,
int member_cnt) {
char errstr[512];
if (err) {
rd_snprintf(errstr, sizeof(errstr),
"Failed to get cluster metadata: %s",
rd_kafka_err2str(err));
goto err;
}
*errstr = '\0';
/* Run assignor */
err = rd_kafka_assignor_run(rkcg, rkas, metadata, members, member_cnt,
errstr, sizeof(errstr));
if (err) {
if (!*errstr)
rd_snprintf(errstr, sizeof(errstr), "%s",
rd_kafka_err2str(err));
goto err;
}
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER, "ASSIGNOR",
"Group \"%s\": \"%s\" assignor run for %d member(s)",
rkcg->rkcg_group_id->str, rkas->rkas_protocol_name->str,
member_cnt);
if (rkas->rkas_protocol == RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE)
rd_kafka_cooperative_protocol_adjust_assignment(rkcg, members,
member_cnt);
rd_kafka_cgrp_set_join_state(rkcg, RD_KAFKA_CGRP_JOIN_STATE_WAIT_SYNC);
rd_kafka_cgrp_set_wait_resp(rkcg, RD_KAFKAP_SyncGroup);
/* Respond to broker with assignment set or error */
rd_kafka_SyncGroupRequest(
rkcg->rkcg_coord, rkcg->rkcg_group_id, rkcg->rkcg_generation_id,
rkcg->rkcg_member_id, rkcg->rkcg_group_instance_id, members,
err ? 0 : member_cnt, RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_SyncGroup, rkcg);
return;
err:
rd_kafka_log(rkcg->rkcg_rk, LOG_ERR, "ASSIGNOR",
"Group \"%s\": failed to run assignor \"%s\" for "
"%d member(s): %s",
rkcg->rkcg_group_id->str, rkas->rkas_protocol_name->str,
member_cnt, errstr);
rd_kafka_cgrp_rejoin(rkcg, "%s assignor failed: %s",
rkas->rkas_protocol_name->str, errstr);
}
/**
* @brief Op callback from handle_JoinGroup
*/
static rd_kafka_op_res_t
rd_kafka_cgrp_assignor_handle_Metadata_op(rd_kafka_t *rk,
rd_kafka_q_t *rkq,
rd_kafka_op_t *rko) {
rd_kafka_cgrp_t *rkcg = rk->rk_cgrp;
if (rko->rko_err == RD_KAFKA_RESP_ERR__DESTROY)
return RD_KAFKA_OP_RES_HANDLED; /* Terminating */
if (rkcg->rkcg_join_state != RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA)
return RD_KAFKA_OP_RES_HANDLED; /* From outdated state */
if (!rkcg->rkcg_group_leader.members) {
rd_kafka_dbg(rk, CGRP, "GRPLEADER",
"Group \"%.*s\": no longer leader: "
"not running assignor",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id));
return RD_KAFKA_OP_RES_HANDLED;
}
rd_kafka_cgrp_assignor_run(rkcg, rkcg->rkcg_assignor, rko->rko_err,
rko->rko_u.metadata.md,
rkcg->rkcg_group_leader.members,
rkcg->rkcg_group_leader.member_cnt);
return RD_KAFKA_OP_RES_HANDLED;
}
/**
* Parse single JoinGroup.Members.MemberMetadata for "consumer" ProtocolType
*
* Protocol definition:
* https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Client-side+Assignment+Proposal
*
* Returns 0 on success or -1 on error.
*/
static int rd_kafka_group_MemberMetadata_consumer_read(
rd_kafka_broker_t *rkb,
rd_kafka_group_member_t *rkgm,
const rd_kafkap_bytes_t *MemberMetadata) {
rd_kafka_buf_t *rkbuf;
int16_t Version;
int32_t subscription_cnt;
rd_kafkap_bytes_t UserData;
const int log_decode_errors = LOG_ERR;
rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR__BAD_MSG;
/* Create a shadow-buffer pointing to the metadata to ease parsing. */
rkbuf = rd_kafka_buf_new_shadow(
MemberMetadata->data, RD_KAFKAP_BYTES_LEN(MemberMetadata), NULL);
/* Protocol parser needs a broker handle to log errors on. */
rkbuf->rkbuf_rkb = rkb;
rd_kafka_broker_keep(rkb);
rd_kafka_buf_read_i16(rkbuf, &Version);
rd_kafka_buf_read_i32(rkbuf, &subscription_cnt);
if (subscription_cnt > 10000 || subscription_cnt <= 0)
goto err;
rkgm->rkgm_subscription =
rd_kafka_topic_partition_list_new(subscription_cnt);
while (subscription_cnt-- > 0) {
rd_kafkap_str_t Topic;
char *topic_name;
rd_kafka_buf_read_str(rkbuf, &Topic);
RD_KAFKAP_STR_DUPA(&topic_name, &Topic);
rd_kafka_topic_partition_list_add(
rkgm->rkgm_subscription, topic_name, RD_KAFKA_PARTITION_UA);
}
rd_kafka_buf_read_bytes(rkbuf, &UserData);
rkgm->rkgm_userdata = rd_kafkap_bytes_copy(&UserData);
const rd_kafka_topic_partition_field_t fields[] = {
RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
RD_KAFKA_TOPIC_PARTITION_FIELD_END};
if (Version >= 1 &&
!(rkgm->rkgm_owned =
rd_kafka_buf_read_topic_partitions(rkbuf, 0, fields)))
goto err;
rd_kafka_buf_destroy(rkbuf);
return 0;
err_parse:
err = rkbuf->rkbuf_err;
err:
rd_rkb_dbg(rkb, CGRP, "MEMBERMETA",
"Failed to parse MemberMetadata for \"%.*s\": %s",
RD_KAFKAP_STR_PR(rkgm->rkgm_member_id),
rd_kafka_err2str(err));
if (rkgm->rkgm_subscription) {
rd_kafka_topic_partition_list_destroy(rkgm->rkgm_subscription);
rkgm->rkgm_subscription = NULL;
}
rd_kafka_buf_destroy(rkbuf);
return -1;
}
/**
* @brief cgrp handler for JoinGroup responses
* opaque must be the cgrp handle.
*
* @locality rdkafka main thread (unless ERR__DESTROY: arbitrary thread)
*/
static void rd_kafka_cgrp_handle_JoinGroup(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
void *opaque) {
rd_kafka_cgrp_t *rkcg = opaque;
const int log_decode_errors = LOG_ERR;
int16_t ErrorCode = 0;
int32_t GenerationId;
rd_kafkap_str_t Protocol, LeaderId;
rd_kafkap_str_t MyMemberId = RD_KAFKAP_STR_INITIALIZER;
int32_t member_cnt;
int actions;
int i_am_leader = 0;
rd_kafka_assignor_t *rkas = NULL;
rd_kafka_cgrp_clear_wait_resp(rkcg, RD_KAFKAP_JoinGroup);
if (err == RD_KAFKA_RESP_ERR__DESTROY ||
rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE)
return; /* Terminating */
if (rkcg->rkcg_join_state != RD_KAFKA_CGRP_JOIN_STATE_WAIT_JOIN) {
rd_kafka_dbg(
rkb->rkb_rk, CGRP, "JOINGROUP",
"JoinGroup response: discarding outdated request "
"(now in join-state %s)",
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
return;
}
if (err) {
ErrorCode = err;
goto err;
}
if (request->rkbuf_reqhdr.ApiVersion >= 2)
rd_kafka_buf_read_throttle_time(rkbuf);
rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
rd_kafka_buf_read_i32(rkbuf, &GenerationId);
rd_kafka_buf_read_str(rkbuf, &Protocol);
rd_kafka_buf_read_str(rkbuf, &LeaderId);
rd_kafka_buf_read_str(rkbuf, &MyMemberId);
rd_kafka_buf_read_i32(rkbuf, &member_cnt);
if (!ErrorCode && RD_KAFKAP_STR_IS_NULL(&Protocol)) {
/* Protocol not set, we will not be able to find
* a matching assignor so error out early. */
ErrorCode = RD_KAFKA_RESP_ERR__BAD_MSG;
} else if (!ErrorCode) {
char *protocol_name;
RD_KAFKAP_STR_DUPA(&protocol_name, &Protocol);
if (!(rkas = rd_kafka_assignor_find(rkcg->rkcg_rk,
protocol_name)) ||
!rkas->rkas_enabled) {
rd_kafka_dbg(rkb->rkb_rk, CGRP, "JOINGROUP",
"Unsupported assignment strategy \"%s\"",
protocol_name);
if (rkcg->rkcg_assignor) {
if (rkcg->rkcg_assignor->rkas_destroy_state_cb)
rkcg->rkcg_assignor
->rkas_destroy_state_cb(
rkcg->rkcg_assignor_state);
rkcg->rkcg_assignor_state = NULL;
rkcg->rkcg_assignor = NULL;
}
ErrorCode = RD_KAFKA_RESP_ERR__UNKNOWN_PROTOCOL;
}
}
rd_kafka_dbg(rkb->rkb_rk, CGRP, "JOINGROUP",
"JoinGroup response: GenerationId %" PRId32
", "
"Protocol %.*s, LeaderId %.*s%s, my MemberId %.*s, "
"member metadata count "
"%" PRId32 ": %s",
GenerationId, RD_KAFKAP_STR_PR(&Protocol),
RD_KAFKAP_STR_PR(&LeaderId),
RD_KAFKAP_STR_LEN(&MyMemberId) &&
!rd_kafkap_str_cmp(&LeaderId, &MyMemberId)
? " (me)"
: "",
RD_KAFKAP_STR_PR(&MyMemberId), member_cnt,
ErrorCode ? rd_kafka_err2str(ErrorCode) : "(no error)");
if (!ErrorCode) {
char *my_member_id;
RD_KAFKAP_STR_DUPA(&my_member_id, &MyMemberId);
rd_kafka_cgrp_set_member_id(rkcg, my_member_id);
rkcg->rkcg_generation_id = GenerationId;
i_am_leader = !rd_kafkap_str_cmp(&LeaderId, &MyMemberId);
} else {
rd_interval_backoff(&rkcg->rkcg_join_intvl, 1000 * 1000);
goto err;
}
if (rkcg->rkcg_assignor && rkcg->rkcg_assignor != rkas) {
if (rkcg->rkcg_assignor->rkas_destroy_state_cb)
rkcg->rkcg_assignor->rkas_destroy_state_cb(
rkcg->rkcg_assignor_state);
rkcg->rkcg_assignor_state = NULL;
}
rkcg->rkcg_assignor = rkas;
if (i_am_leader) {
rd_kafka_group_member_t *members;
int i;
int sub_cnt = 0;
rd_list_t topics;
rd_kafka_op_t *rko;
rd_kafka_dbg(rkb->rkb_rk, CGRP, "JOINGROUP",
"I am elected leader for group \"%s\" "
"with %" PRId32 " member(s)",
rkcg->rkcg_group_id->str, member_cnt);
if (member_cnt > 100000) {
err = RD_KAFKA_RESP_ERR__BAD_MSG;
goto err;
}
rd_list_init(&topics, member_cnt, rd_free);
members = rd_calloc(member_cnt, sizeof(*members));
for (i = 0; i < member_cnt; i++) {
rd_kafkap_str_t MemberId;
rd_kafkap_bytes_t MemberMetadata;
rd_kafka_group_member_t *rkgm;
rd_kafkap_str_t GroupInstanceId =
RD_KAFKAP_STR_INITIALIZER;
rd_kafka_buf_read_str(rkbuf, &MemberId);
if (request->rkbuf_reqhdr.ApiVersion >= 5)
rd_kafka_buf_read_str(rkbuf, &GroupInstanceId);
rd_kafka_buf_read_bytes(rkbuf, &MemberMetadata);
rkgm = &members[sub_cnt];
rkgm->rkgm_member_id = rd_kafkap_str_copy(&MemberId);
rkgm->rkgm_group_instance_id =
rd_kafkap_str_copy(&GroupInstanceId);
rd_list_init(&rkgm->rkgm_eligible, 0, NULL);
rkgm->rkgm_generation = -1;
if (rd_kafka_group_MemberMetadata_consumer_read(
rkb, rkgm, &MemberMetadata)) {
/* Failed to parse this member's metadata,
* ignore it. */
} else {
sub_cnt++;
rkgm->rkgm_assignment =
rd_kafka_topic_partition_list_new(
rkgm->rkgm_subscription->cnt);
rd_kafka_topic_partition_list_get_topic_names(
rkgm->rkgm_subscription, &topics,
0 /*dont include regex*/);
}
}
/* FIXME: What to do if parsing failed for some/all members?
* It is a sign of incompatibility. */
rd_kafka_cgrp_group_leader_reset(rkcg,
"JoinGroup response clean-up");
rd_kafka_assert(NULL, rkcg->rkcg_group_leader.members == NULL);
rkcg->rkcg_group_leader.members = members;
rkcg->rkcg_group_leader.member_cnt = sub_cnt;
rd_kafka_cgrp_set_join_state(
rkcg, RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA);
/* The assignor will need metadata so fetch it asynchronously
* and run the assignor when we get a reply.
* Create a callback op that the generic metadata code
* will trigger when metadata has been parsed. */
rko = rd_kafka_op_new_cb(
rkcg->rkcg_rk, RD_KAFKA_OP_METADATA,
rd_kafka_cgrp_assignor_handle_Metadata_op);
rd_kafka_op_set_replyq(rko, rkcg->rkcg_ops, NULL);
rd_kafka_MetadataRequest(
rkb, &topics, "partition assignor",
rd_false /*!allow_auto_create*/,
/* cgrp_update=false:
* Since the subscription list may not be identical
* across all members of the group and thus the
* Metadata response may not be identical to this
* consumer's subscription list, we want to
* avoid triggering a rejoin or error propagation
* on receiving the response since some topics
* may be missing. */
rd_false, rko);
rd_list_destroy(&topics);
} else {
rd_kafka_cgrp_set_join_state(
rkcg, RD_KAFKA_CGRP_JOIN_STATE_WAIT_SYNC);
rd_kafka_cgrp_set_wait_resp(rkcg, RD_KAFKAP_SyncGroup);
rd_kafka_SyncGroupRequest(
rkb, rkcg->rkcg_group_id, rkcg->rkcg_generation_id,
rkcg->rkcg_member_id, rkcg->rkcg_group_instance_id, NULL, 0,
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_SyncGroup, rkcg);
}
err:
actions = rd_kafka_err_action(
rkb, ErrorCode, request, RD_KAFKA_ERR_ACTION_IGNORE,
RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID,
RD_KAFKA_ERR_ACTION_IGNORE, RD_KAFKA_RESP_ERR_MEMBER_ID_REQUIRED,
RD_KAFKA_ERR_ACTION_IGNORE, RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION,
RD_KAFKA_ERR_ACTION_PERMANENT, RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID,
RD_KAFKA_ERR_ACTION_END);
if (actions & RD_KAFKA_ERR_ACTION_REFRESH) {
/* Re-query for coordinator */
rd_kafka_cgrp_op(rkcg, NULL, RD_KAFKA_NO_REPLYQ,
RD_KAFKA_OP_COORD_QUERY, ErrorCode);
}
/* No need for retries here since the join is intervalled,
* see rkcg_join_intvl */
if (ErrorCode) {
if (ErrorCode == RD_KAFKA_RESP_ERR__DESTROY)
return; /* Termination */
if (ErrorCode == RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID) {
rd_kafka_set_fatal_error(rkcg->rkcg_rk, ErrorCode,
"Fatal consumer error: %s",
rd_kafka_err2str(ErrorCode));
ErrorCode = RD_KAFKA_RESP_ERR__FATAL;
} else if (actions & RD_KAFKA_ERR_ACTION_PERMANENT)
rd_kafka_consumer_err(
rkcg->rkcg_q, rd_kafka_broker_id(rkb), ErrorCode, 0,
NULL, NULL, RD_KAFKA_OFFSET_INVALID,
"JoinGroup failed: %s",
rd_kafka_err2str(ErrorCode));
if (ErrorCode == RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID)
rd_kafka_cgrp_set_member_id(rkcg, "");
else if (ErrorCode == RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION)
rkcg->rkcg_generation_id = -1;
else if (ErrorCode == RD_KAFKA_RESP_ERR_MEMBER_ID_REQUIRED) {
/* KIP-394 requires member.id on initial join
* group request */
char *my_member_id;
RD_KAFKAP_STR_DUPA(&my_member_id, &MyMemberId);
rd_kafka_cgrp_set_member_id(rkcg, my_member_id);
/* Skip the join backoff */
rd_interval_reset(&rkcg->rkcg_join_intvl);
}
if (rd_kafka_cgrp_rebalance_protocol(rkcg) ==
RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE &&
(ErrorCode == RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION ||
ErrorCode == RD_KAFKA_RESP_ERR_MEMBER_ID_REQUIRED))
rd_kafka_cgrp_revoke_all_rejoin(
rkcg, rd_true /*assignment is lost*/,
rd_true /*this consumer is initiating*/,
"JoinGroup error");
else
rd_kafka_cgrp_rejoin(rkcg, "JoinGroup error: %s",
rd_kafka_err2str(ErrorCode));
}
return;
err_parse:
ErrorCode = rkbuf->rkbuf_err;
goto err;
}
/**
* @brief Check subscription against requested Metadata.
*/
static rd_kafka_op_res_t rd_kafka_cgrp_handle_Metadata_op(rd_kafka_t *rk,
rd_kafka_q_t *rkq,
rd_kafka_op_t *rko) {
rd_kafka_cgrp_t *rkcg = rk->rk_cgrp;
if (rko->rko_err == RD_KAFKA_RESP_ERR__DESTROY)
return RD_KAFKA_OP_RES_HANDLED; /* Terminating */
rd_kafka_cgrp_metadata_update_check(rkcg, rd_false /*dont rejoin*/);
return RD_KAFKA_OP_RES_HANDLED;
}
/**
* @brief (Async) Refresh metadata (for cgrp's needs)
*
* @returns 1 if metadata refresh was requested, or 0 if metadata is
* up to date, or -1 if no broker is available for metadata requests.
*
* @locks none
* @locality rdkafka main thread
*/
static int rd_kafka_cgrp_metadata_refresh(rd_kafka_cgrp_t *rkcg,
int *metadata_agep,
const char *reason) {
rd_kafka_t *rk = rkcg->rkcg_rk;
rd_kafka_op_t *rko;
rd_list_t topics;
rd_kafka_resp_err_t err;
rd_list_init(&topics, 8, rd_free);
/* Insert all non-wildcard topics in cache. */
rd_kafka_metadata_cache_hint_rktparlist(
rkcg->rkcg_rk, rkcg->rkcg_subscription, NULL, 0 /*dont replace*/);
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION) {
/* For wildcard subscriptions make sure the
* cached full metadata isn't too old. */
int metadata_age = -1;
if (rk->rk_ts_full_metadata)
metadata_age =
(int)(rd_clock() - rk->rk_ts_full_metadata) / 1000;
*metadata_agep = metadata_age;
if (metadata_age != -1 &&
metadata_age <= rk->rk_conf.metadata_max_age_ms) {
rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA,
"CGRPMETADATA",
"%s: metadata for wildcard subscription "
"is up to date (%dms old)",
reason, *metadata_agep);
rd_list_destroy(&topics);
return 0; /* Up-to-date */
}
} else {
/* Check that all subscribed topics are in the cache. */
int r;
rd_kafka_topic_partition_list_get_topic_names(
rkcg->rkcg_subscription, &topics, 0 /*no regexps*/);
rd_kafka_rdlock(rk);
r = rd_kafka_metadata_cache_topics_count_exists(rk, &topics,
metadata_agep);
rd_kafka_rdunlock(rk);
if (r == rd_list_cnt(&topics)) {
rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA,
"CGRPMETADATA",
"%s: metadata for subscription "
"is up to date (%dms old)",
reason, *metadata_agep);
rd_list_destroy(&topics);
return 0; /* Up-to-date and all topics exist. */
}
rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA, "CGRPMETADATA",
"%s: metadata for subscription "
"only available for %d/%d topics (%dms old)",
reason, r, rd_list_cnt(&topics), *metadata_agep);
}
/* Async request, result will be triggered from
* rd_kafka_parse_metadata(). */
rko = rd_kafka_op_new_cb(rkcg->rkcg_rk, RD_KAFKA_OP_METADATA,
rd_kafka_cgrp_handle_Metadata_op);
rd_kafka_op_set_replyq(rko, rkcg->rkcg_ops, 0);
err = rd_kafka_metadata_request(rkcg->rkcg_rk, NULL, &topics,
rd_false /*!allow auto create */,
rd_true /*cgrp_update*/, reason, rko);
if (err) {
rd_kafka_dbg(rk, CGRP | RD_KAFKA_DBG_METADATA, "CGRPMETADATA",
"%s: need to refresh metadata (%dms old) "
"but no usable brokers available: %s",
reason, *metadata_agep, rd_kafka_err2str(err));
rd_kafka_op_destroy(rko);
}
rd_list_destroy(&topics);
return err ? -1 : 1;
}
static void rd_kafka_cgrp_join(rd_kafka_cgrp_t *rkcg) {
int metadata_age;
if (rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP ||
rkcg->rkcg_join_state != RD_KAFKA_CGRP_JOIN_STATE_INIT ||
rd_kafka_cgrp_awaiting_response(rkcg))
return;
/* On max.poll.interval.ms failure, do not rejoin group until the
* application has called poll. */
if ((rkcg->rkcg_flags & RD_KAFKA_CGRP_F_MAX_POLL_EXCEEDED) &&
rd_kafka_max_poll_exceeded(rkcg->rkcg_rk))
return;
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_MAX_POLL_EXCEEDED;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "JOIN",
"Group \"%.*s\": join with %d subscribed topic(s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_list_cnt(rkcg->rkcg_subscribed_topics));
/* See if we need to query metadata to continue:
* - if subscription contains wildcards:
* * query all topics in cluster
*
* - if subscription does not contain wildcards but
* some topics are missing from the local metadata cache:
* * query subscribed topics (all cached ones)
*
* - otherwise:
* * rely on topic metadata cache
*/
/* We need up-to-date full metadata to continue,
* refresh metadata if necessary. */
if (rd_kafka_cgrp_metadata_refresh(rkcg, &metadata_age,
"consumer join") == 1) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER,
"JOIN",
"Group \"%.*s\": "
"postponing join until up-to-date "
"metadata is available",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id));
rd_assert(
rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT ||
/* Possible via rd_kafka_cgrp_modify_subscription */
rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY);
rd_kafka_cgrp_set_join_state(
rkcg, RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA);
return; /* ^ async call */
}
if (rd_list_empty(rkcg->rkcg_subscribed_topics))
rd_kafka_cgrp_metadata_update_check(rkcg,
rd_false /*dont join*/);
if (rd_list_empty(rkcg->rkcg_subscribed_topics)) {
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER, "JOIN",
"Group \"%.*s\": "
"no matching topics based on %dms old metadata: "
"next metadata refresh in %dms",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), metadata_age,
rkcg->rkcg_rk->rk_conf.metadata_refresh_interval_ms -
metadata_age);
return;
}
rd_rkb_dbg(
rkcg->rkcg_curr_coord, CONSUMER | RD_KAFKA_DBG_CGRP, "JOIN",
"Joining group \"%.*s\" with %d subscribed topic(s) and "
"member id \"%.*s\"",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_list_cnt(rkcg->rkcg_subscribed_topics),
rkcg->rkcg_member_id ? RD_KAFKAP_STR_LEN(rkcg->rkcg_member_id) : 0,
rkcg->rkcg_member_id ? rkcg->rkcg_member_id->str : "");
rd_kafka_cgrp_set_join_state(rkcg, RD_KAFKA_CGRP_JOIN_STATE_WAIT_JOIN);
rd_kafka_cgrp_set_wait_resp(rkcg, RD_KAFKAP_JoinGroup);
rd_kafka_JoinGroupRequest(
rkcg->rkcg_coord, rkcg->rkcg_group_id, rkcg->rkcg_member_id,
rkcg->rkcg_group_instance_id,
rkcg->rkcg_rk->rk_conf.group_protocol_type,
rkcg->rkcg_subscribed_topics, RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_handle_JoinGroup, rkcg);
}
/**
* Rejoin group on update to effective subscribed topics list
*/
static void rd_kafka_cgrp_revoke_rejoin(rd_kafka_cgrp_t *rkcg,
const char *reason) {
/*
* Clean-up group leader duties, if any.
*/
rd_kafka_cgrp_group_leader_reset(rkcg, "group (re)join");
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP, "REJOIN",
"Group \"%.*s\" (re)joining in join-state %s "
"with %d assigned partition(s): %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
rkcg->rkcg_group_assignment ? rkcg->rkcg_group_assignment->cnt : 0,
reason);
rd_kafka_cgrp_revoke_all_rejoin(rkcg, rd_false /*not lost*/,
rd_true /*initiating*/, reason);
}
/**
* @brief Update the effective list of subscribed topics.
*
* Set \p tinfos to NULL to clear the list.
*
* @param tinfos rd_list_t(rd_kafka_topic_info_t *): new effective topic list
*
* @returns true on change, else false.
*
* @remark Takes ownership of \p tinfos
*/
static rd_bool_t rd_kafka_cgrp_update_subscribed_topics(rd_kafka_cgrp_t *rkcg,
rd_list_t *tinfos) {
rd_kafka_topic_info_t *tinfo;
int i;
if (!tinfos) {
if (!rd_list_empty(rkcg->rkcg_subscribed_topics))
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "SUBSCRIPTION",
"Group \"%.*s\": "
"clearing subscribed topics list (%d)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_list_cnt(rkcg->rkcg_subscribed_topics));
tinfos = rd_list_new(0, (void *)rd_kafka_topic_info_destroy);
} else {
if (rd_list_cnt(tinfos) == 0)
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "SUBSCRIPTION",
"Group \"%.*s\": "
"no topics in metadata matched "
"subscription",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id));
}
/* Sort for comparison */
rd_list_sort(tinfos, rd_kafka_topic_info_cmp);
/* Compare to existing to see if anything changed. */
if (!rd_list_cmp(rkcg->rkcg_subscribed_topics, tinfos,
rd_kafka_topic_info_cmp)) {
/* No change */
rd_list_destroy(tinfos);
return rd_false;
}
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_METADATA, "SUBSCRIPTION",
"Group \"%.*s\": effective subscription list changed "
"from %d to %d topic(s):",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_list_cnt(rkcg->rkcg_subscribed_topics), rd_list_cnt(tinfos));
RD_LIST_FOREACH(tinfo, tinfos, i)
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_METADATA,
"SUBSCRIPTION", " Topic %s with %d partition(s)",
tinfo->topic, tinfo->partition_cnt);
rd_list_destroy(rkcg->rkcg_subscribed_topics);
rkcg->rkcg_subscribed_topics = tinfos;
return rd_true;
}
/**
* @brief Handle Heartbeat response.
*/
void rd_kafka_cgrp_handle_Heartbeat(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
void *opaque) {
rd_kafka_cgrp_t *rkcg = rk->rk_cgrp;
const int log_decode_errors = LOG_ERR;
int16_t ErrorCode = 0;
int actions = 0;
if (err == RD_KAFKA_RESP_ERR__DESTROY)
return;
rd_dassert(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT);
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT;
rkcg->rkcg_last_heartbeat_err = RD_KAFKA_RESP_ERR_NO_ERROR;
if (err)
goto err;
if (request->rkbuf_reqhdr.ApiVersion >= 1)
rd_kafka_buf_read_throttle_time(rkbuf);
rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
if (ErrorCode) {
err = ErrorCode;
goto err;
}
rd_kafka_cgrp_update_session_timeout(
rkcg, rd_false /*don't update if session has expired*/);
return;
err_parse:
err = rkbuf->rkbuf_err;
err:
rkcg->rkcg_last_heartbeat_err = err;
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP, "HEARTBEAT",
"Group \"%s\" heartbeat error response in "
"state %s (join-state %s, %d partition(s) assigned): %s",
rkcg->rkcg_group_id->str,
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
rkcg->rkcg_group_assignment ? rkcg->rkcg_group_assignment->cnt : 0,
rd_kafka_err2str(err));
if (rkcg->rkcg_join_state <= RD_KAFKA_CGRP_JOIN_STATE_WAIT_SYNC) {
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP, "HEARTBEAT",
"Heartbeat response: discarding outdated "
"request (now in join-state %s)",
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
return;
}
switch (err) {
case RD_KAFKA_RESP_ERR__DESTROY:
/* quick cleanup */
return;
case RD_KAFKA_RESP_ERR_NOT_COORDINATOR_FOR_GROUP:
case RD_KAFKA_RESP_ERR_GROUP_COORDINATOR_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR__TRANSPORT:
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER, "HEARTBEAT",
"Heartbeat failed due to coordinator (%s) "
"no longer available: %s: "
"re-querying for coordinator",
rkcg->rkcg_curr_coord
? rd_kafka_broker_name(rkcg->rkcg_curr_coord)
: "none",
rd_kafka_err2str(err));
/* Remain in joined state and keep querying for coordinator */
actions = RD_KAFKA_ERR_ACTION_REFRESH;
break;
case RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS:
rd_kafka_cgrp_update_session_timeout(
rkcg, rd_false /*don't update if session has expired*/);
/* No further action if already rebalancing */
if (RD_KAFKA_CGRP_WAIT_ASSIGN_CALL(rkcg))
return;
rd_kafka_cgrp_group_is_rebalancing(rkcg);
return;
case RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID:
rd_kafka_cgrp_set_member_id(rkcg, "");
rd_kafka_cgrp_revoke_all_rejoin_maybe(rkcg, rd_true /*lost*/,
rd_true /*initiating*/,
"resetting member-id");
return;
case RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION:
rkcg->rkcg_generation_id = -1;
rd_kafka_cgrp_revoke_all_rejoin_maybe(rkcg, rd_true /*lost*/,
rd_true /*initiating*/,
"illegal generation");
return;
case RD_KAFKA_RESP_ERR_FENCED_INSTANCE_ID:
rd_kafka_set_fatal_error(rkcg->rkcg_rk, err,
"Fatal consumer error: %s",
rd_kafka_err2str(err));
rd_kafka_cgrp_revoke_all_rejoin_maybe(
rkcg, rd_true, /*assignment lost*/
rd_true, /*initiating*/
"consumer fenced by "
"newer instance");
return;
default:
actions = rd_kafka_err_action(rkb, err, request,
RD_KAFKA_ERR_ACTION_END);
break;
}
if (actions & RD_KAFKA_ERR_ACTION_REFRESH) {
/* Re-query for coordinator */
rd_kafka_cgrp_coord_query(rkcg, rd_kafka_err2str(err));
}
if (actions & RD_KAFKA_ERR_ACTION_RETRY &&
rd_kafka_buf_retry(rkb, request)) {
/* Retry */
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT;
return;
}
}
/**
* @brief Send Heartbeat
*/
static void rd_kafka_cgrp_heartbeat(rd_kafka_cgrp_t *rkcg) {
/* Don't send heartbeats if max.poll.interval.ms was exceeded */
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_MAX_POLL_EXCEEDED)
return;
/* Skip heartbeat if we have one in transit */
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT)
return;
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT;
rd_kafka_HeartbeatRequest(
rkcg->rkcg_coord, rkcg->rkcg_group_id, rkcg->rkcg_generation_id,
rkcg->rkcg_member_id, rkcg->rkcg_group_instance_id,
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0), rd_kafka_cgrp_handle_Heartbeat,
NULL);
}
/**
* Cgrp is now terminated: decommission it and signal back to application.
*/
static void rd_kafka_cgrp_terminated(rd_kafka_cgrp_t *rkcg) {
if (rd_atomic32_get(&rkcg->rkcg_terminated))
return; /* terminated() may be called multiple times,
* make sure to only terminate once. */
rd_kafka_cgrp_group_assignment_set(rkcg, NULL);
rd_kafka_assert(NULL, !rd_kafka_assignment_in_progress(rkcg->rkcg_rk));
rd_kafka_assert(NULL, !rkcg->rkcg_group_assignment);
rd_kafka_assert(NULL, rkcg->rkcg_rk->rk_consumer.wait_commit_cnt == 0);
rd_kafka_assert(NULL, rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_TERM);
rd_kafka_timer_stop(&rkcg->rkcg_rk->rk_timers,
&rkcg->rkcg_offset_commit_tmr, 1 /*lock*/);
rd_kafka_q_purge(rkcg->rkcg_wait_coord_q);
/* Disable and empty ops queue since there will be no
* (broker) thread serving it anymore after the unassign_broker
* below.
* This prevents hang on destroy where responses are enqueued on
* rkcg_ops without anything serving the queue. */
rd_kafka_q_disable(rkcg->rkcg_ops);
rd_kafka_q_purge(rkcg->rkcg_ops);
if (rkcg->rkcg_curr_coord)
rd_kafka_cgrp_coord_clear_broker(rkcg);
if (rkcg->rkcg_coord) {
rd_kafka_broker_destroy(rkcg->rkcg_coord);
rkcg->rkcg_coord = NULL;
}
rd_atomic32_set(&rkcg->rkcg_terminated, rd_true);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPTERM",
"Consumer group sub-system terminated%s",
rkcg->rkcg_reply_rko ? " (will enqueue reply)" : "");
if (rkcg->rkcg_reply_rko) {
/* Signal back to application. */
rd_kafka_replyq_enq(&rkcg->rkcg_reply_rko->rko_replyq,
rkcg->rkcg_reply_rko, 0);
rkcg->rkcg_reply_rko = NULL;
}
/* Remove cgrp application queue forwarding, if any. */
rd_kafka_q_fwd_set(rkcg->rkcg_q, NULL);
}
/**
* If a cgrp is terminating and all outstanding ops are now finished
* then progress to final termination and return 1.
* Else returns 0.
*/
static RD_INLINE int rd_kafka_cgrp_try_terminate(rd_kafka_cgrp_t *rkcg) {
if (rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_TERM)
return 1;
if (likely(!(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE)))
return 0;
/* Check if wait-coord queue has timed out. */
if (rd_kafka_q_len(rkcg->rkcg_wait_coord_q) > 0 &&
rkcg->rkcg_ts_terminate +
(rkcg->rkcg_rk->rk_conf.group_session_timeout_ms * 1000) <
rd_clock()) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPTERM",
"Group \"%s\": timing out %d op(s) in "
"wait-for-coordinator queue",
rkcg->rkcg_group_id->str,
rd_kafka_q_len(rkcg->rkcg_wait_coord_q));
rd_kafka_q_disable(rkcg->rkcg_wait_coord_q);
if (rd_kafka_q_concat(rkcg->rkcg_ops,
rkcg->rkcg_wait_coord_q) == -1) {
/* ops queue shut down, purge coord queue */
rd_kafka_q_purge(rkcg->rkcg_wait_coord_q);
}
}
if (!RD_KAFKA_CGRP_WAIT_ASSIGN_CALL(rkcg) &&
rd_list_empty(&rkcg->rkcg_toppars) &&
!rd_kafka_assignment_in_progress(rkcg->rkcg_rk) &&
rkcg->rkcg_rk->rk_consumer.wait_commit_cnt == 0 &&
!(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WAIT_LEAVE)) {
/* Since we might be deep down in a 'rko' handler
* called from cgrp_op_serve() we cant call terminated()
* directly since it will decommission the rkcg_ops queue
* that might be locked by intermediate functions.
* Instead set the TERM state and let the cgrp terminate
* at its own discretion. */
rd_kafka_cgrp_set_state(rkcg, RD_KAFKA_CGRP_STATE_TERM);
return 1;
} else {
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP, "CGRPTERM",
"Group \"%s\": "
"waiting for %s%d toppar(s), "
"%s"
"%d commit(s)%s%s%s (state %s, join-state %s) "
"before terminating",
rkcg->rkcg_group_id->str,
RD_KAFKA_CGRP_WAIT_ASSIGN_CALL(rkcg) ? "assign call, " : "",
rd_list_cnt(&rkcg->rkcg_toppars),
rd_kafka_assignment_in_progress(rkcg->rkcg_rk)
? "assignment in progress, "
: "",
rkcg->rkcg_rk->rk_consumer.wait_commit_cnt,
(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WAIT_LEAVE)
? ", wait-leave,"
: "",
rkcg->rkcg_rebalance_rejoin ? ", rebalance_rejoin," : "",
(rkcg->rkcg_rebalance_incr_assignment != NULL)
? ", rebalance_incr_assignment,"
: "",
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
return 0;
}
}
/**
* @brief Add partition to this cgrp management
*
* @locks none
*/
static void rd_kafka_cgrp_partition_add(rd_kafka_cgrp_t *rkcg,
rd_kafka_toppar_t *rktp) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "PARTADD",
"Group \"%s\": add %s [%" PRId32 "]",
rkcg->rkcg_group_id->str, rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition);
rd_kafka_toppar_lock(rktp);
rd_assert(!(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_CGRP));
rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_ON_CGRP;
rd_kafka_toppar_unlock(rktp);
rd_kafka_toppar_keep(rktp);
rd_list_add(&rkcg->rkcg_toppars, rktp);
}
/**
* @brief Remove partition from this cgrp management
*
* @locks none
*/
static void rd_kafka_cgrp_partition_del(rd_kafka_cgrp_t *rkcg,
rd_kafka_toppar_t *rktp) {
int cnt = 0, barrier_cnt = 0, message_cnt = 0, other_cnt = 0;
rd_kafka_op_t *rko;
rd_kafka_q_t *rkq;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "PARTDEL",
"Group \"%s\": delete %s [%" PRId32 "]",
rkcg->rkcg_group_id->str, rktp->rktp_rkt->rkt_topic->str,
rktp->rktp_partition);
rd_kafka_toppar_lock(rktp);
rd_assert(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_ON_CGRP);
rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_ON_CGRP;
if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_REMOVE) {
/* Partition is being removed from the cluster and it's stopped,
* so rktp->rktp_fetchq->rkq_fwdq is NULL.
* Purge remaining operations in rktp->rktp_fetchq->rkq_q,
* while holding lock, to avoid circular references */
rkq = rktp->rktp_fetchq;
mtx_lock(&rkq->rkq_lock);
rd_assert(!rkq->rkq_fwdq);
rko = TAILQ_FIRST(&rkq->rkq_q);
while (rko) {
if (rko->rko_type != RD_KAFKA_OP_BARRIER &&
rko->rko_type != RD_KAFKA_OP_FETCH) {
rd_kafka_log(
rkcg->rkcg_rk, LOG_WARNING, "PARTDEL",
"Purging toppar fetch queue buffer op"
"with unexpected type: %s",
rd_kafka_op2str(rko->rko_type));
}
if (rko->rko_type == RD_KAFKA_OP_BARRIER)
barrier_cnt++;
else if (rko->rko_type == RD_KAFKA_OP_FETCH)
message_cnt++;
else
other_cnt++;
rko = TAILQ_NEXT(rko, rko_link);
cnt++;
}
mtx_unlock(&rkq->rkq_lock);
if (cnt) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "PARTDEL",
"Purge toppar fetch queue buffer "
"containing %d op(s) "
"(%d barrier(s), %d message(s), %d other)"
" to avoid "
"circular references",
cnt, barrier_cnt, message_cnt, other_cnt);
rd_kafka_q_purge(rktp->rktp_fetchq);
} else {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "PARTDEL",
"Not purging toppar fetch queue buffer."
" No ops present in the buffer.");
}
}
rd_kafka_toppar_unlock(rktp);
rd_list_remove(&rkcg->rkcg_toppars, rktp);
rd_kafka_toppar_destroy(rktp); /* refcnt from _add above */
rd_kafka_cgrp_try_terminate(rkcg);
}
/**
* @brief Defer offset commit (rko) until coordinator is available.
*
* @returns 1 if the rko was deferred or 0 if the defer queue is disabled
* or rko already deferred.
*/
static int rd_kafka_cgrp_defer_offset_commit(rd_kafka_cgrp_t *rkcg,
rd_kafka_op_t *rko,
const char *reason) {
/* wait_coord_q is disabled session.timeout.ms after
* group close() has been initated. */
if (rko->rko_u.offset_commit.ts_timeout != 0 ||
!rd_kafka_q_ready(rkcg->rkcg_wait_coord_q))
return 0;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COMMIT",
"Group \"%s\": "
"unable to OffsetCommit in state %s: %s: "
"coordinator (%s) is unavailable: "
"retrying later",
rkcg->rkcg_group_id->str,
rd_kafka_cgrp_state_names[rkcg->rkcg_state], reason,
rkcg->rkcg_curr_coord
? rd_kafka_broker_name(rkcg->rkcg_curr_coord)
: "none");
rko->rko_flags |= RD_KAFKA_OP_F_REPROCESS;
rko->rko_u.offset_commit.ts_timeout =
rd_clock() +
(rkcg->rkcg_rk->rk_conf.group_session_timeout_ms * 1000);
rd_kafka_q_enq(rkcg->rkcg_wait_coord_q, rko);
return 1;
}
/**
* @brief Update the committed offsets for the partitions in \p offsets,
*
* @remark \p offsets may be NULL if \p err is set
* @returns the number of partitions with errors encountered
*/
static int rd_kafka_cgrp_update_committed_offsets(
rd_kafka_cgrp_t *rkcg,
rd_kafka_resp_err_t err,
rd_kafka_topic_partition_list_t *offsets) {
int i;
int errcnt = 0;
/* Update toppars' committed offset or global error */
for (i = 0; offsets && i < offsets->cnt; i++) {
rd_kafka_topic_partition_t *rktpar = &offsets->elems[i];
rd_kafka_toppar_t *rktp;
/* Ignore logical offsets since they were never
* sent to the broker. */
if (RD_KAFKA_OFFSET_IS_LOGICAL(rktpar->offset))
continue;
/* Propagate global error to all partitions that don't have
* explicit error set. */
if (err && !rktpar->err)
rktpar->err = err;
if (rktpar->err) {
rd_kafka_dbg(rkcg->rkcg_rk, TOPIC, "OFFSET",
"OffsetCommit failed for "
"%s [%" PRId32
"] at offset "
"%" PRId64 " in join-state %s: %s",
rktpar->topic, rktpar->partition,
rktpar->offset,
rd_kafka_cgrp_join_state_names
[rkcg->rkcg_join_state],
rd_kafka_err2str(rktpar->err));
errcnt++;
continue;
}
rktp = rd_kafka_topic_partition_get_toppar(rkcg->rkcg_rk,
rktpar, rd_false);
if (!rktp)
continue;
rd_kafka_toppar_lock(rktp);
rktp->rktp_committed_pos =
rd_kafka_topic_partition_get_fetch_pos(rktpar);
rd_kafka_toppar_unlock(rktp);
rd_kafka_toppar_destroy(rktp); /* from get_toppar() */
}
return errcnt;
}
/**
* @brief Propagate OffsetCommit results.
*
* @param rko_orig The original rko that triggered the commit, this is used
* to propagate the result.
* @param err Is the aggregated request-level error, or ERR_NO_ERROR.
* @param errcnt Are the number of partitions in \p offsets that failed
* offset commit.
*/
static void rd_kafka_cgrp_propagate_commit_result(
rd_kafka_cgrp_t *rkcg,
rd_kafka_op_t *rko_orig,
rd_kafka_resp_err_t err,
int errcnt,
rd_kafka_topic_partition_list_t *offsets) {
const rd_kafka_t *rk = rkcg->rkcg_rk;
int offset_commit_cb_served = 0;
/* If no special callback is set but a offset_commit_cb has
* been set in conf then post an event for the latter. */
if (!rko_orig->rko_u.offset_commit.cb && rk->rk_conf.offset_commit_cb) {
rd_kafka_op_t *rko_reply = rd_kafka_op_new_reply(rko_orig, err);
rd_kafka_op_set_prio(rko_reply, RD_KAFKA_PRIO_HIGH);
if (offsets)
rko_reply->rko_u.offset_commit.partitions =
rd_kafka_topic_partition_list_copy(offsets);
rko_reply->rko_u.offset_commit.cb =
rk->rk_conf.offset_commit_cb;
rko_reply->rko_u.offset_commit.opaque = rk->rk_conf.opaque;
rd_kafka_q_enq(rk->rk_rep, rko_reply);
offset_commit_cb_served++;
}
/* Enqueue reply to requester's queue, if any. */
if (rko_orig->rko_replyq.q) {
rd_kafka_op_t *rko_reply = rd_kafka_op_new_reply(rko_orig, err);
rd_kafka_op_set_prio(rko_reply, RD_KAFKA_PRIO_HIGH);
/* Copy offset & partitions & callbacks to reply op */
rko_reply->rko_u.offset_commit = rko_orig->rko_u.offset_commit;
if (offsets)
rko_reply->rko_u.offset_commit.partitions =
rd_kafka_topic_partition_list_copy(offsets);
if (rko_reply->rko_u.offset_commit.reason)
rko_reply->rko_u.offset_commit.reason =
rd_strdup(rko_reply->rko_u.offset_commit.reason);
rd_kafka_replyq_enq(&rko_orig->rko_replyq, rko_reply, 0);
offset_commit_cb_served++;
}
if (!offset_commit_cb_served && offsets &&
(errcnt > 0 || (err != RD_KAFKA_RESP_ERR_NO_ERROR &&
err != RD_KAFKA_RESP_ERR__NO_OFFSET))) {
/* If there is no callback or handler for this (auto)
* commit then log an error (#1043) */
char tmp[512];
rd_kafka_topic_partition_list_str(
offsets, tmp, sizeof(tmp),
/* Print per-partition errors unless there was a
* request-level error. */
RD_KAFKA_FMT_F_OFFSET |
(errcnt ? RD_KAFKA_FMT_F_ONLY_ERR : 0));
rd_kafka_log(
rkcg->rkcg_rk, LOG_WARNING, "COMMITFAIL",
"Offset commit (%s) failed "
"for %d/%d partition(s) in join-state %s: "
"%s%s%s",
rko_orig->rko_u.offset_commit.reason,
errcnt ? errcnt : offsets->cnt, offsets->cnt,
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
errcnt ? rd_kafka_err2str(err) : "", errcnt ? ": " : "",
tmp);
}
}
/**
* @brief Handle OffsetCommitResponse
* Takes the original 'rko' as opaque argument.
* @remark \p rkb, rkbuf, and request may be NULL in a number of
* error cases (e.g., _NO_OFFSET, _WAIT_COORD)
*/
static void rd_kafka_cgrp_op_handle_OffsetCommit(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
void *opaque) {
rd_kafka_cgrp_t *rkcg = rk->rk_cgrp;
rd_kafka_op_t *rko_orig = opaque;
rd_kafka_topic_partition_list_t *offsets =
rko_orig->rko_u.offset_commit.partitions; /* maybe NULL */
int errcnt;
RD_KAFKA_OP_TYPE_ASSERT(rko_orig, RD_KAFKA_OP_OFFSET_COMMIT);
err = rd_kafka_handle_OffsetCommit(rk, rkb, err, rkbuf, request,
offsets, rd_false);
/* Suppress empty commit debug logs if allowed */
if (err != RD_KAFKA_RESP_ERR__NO_OFFSET ||
!rko_orig->rko_u.offset_commit.silent_empty) {
if (rkb)
rd_rkb_dbg(rkb, CGRP, "COMMIT",
"OffsetCommit for %d partition(s) in "
"join-state %s: "
"%s: returned: %s",
offsets ? offsets->cnt : -1,
rd_kafka_cgrp_join_state_names
[rkcg->rkcg_join_state],
rko_orig->rko_u.offset_commit.reason,
rd_kafka_err2str(err));
else
rd_kafka_dbg(rk, CGRP, "COMMIT",
"OffsetCommit for %d partition(s) in "
"join-state "
"%s: %s: "
"returned: %s",
offsets ? offsets->cnt : -1,
rd_kafka_cgrp_join_state_names
[rkcg->rkcg_join_state],
rko_orig->rko_u.offset_commit.reason,
rd_kafka_err2str(err));
}
/*
* Error handling
*/
switch (err) {
case RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID:
/* Revoke assignment and rebalance on unknown member */
rd_kafka_cgrp_set_member_id(rk->rk_cgrp, "");
rd_kafka_cgrp_revoke_all_rejoin_maybe(
rkcg, rd_true /*assignment is lost*/,
rd_true /*this consumer is initiating*/,
"OffsetCommit error: Unknown member");
break;
case RD_KAFKA_RESP_ERR_ILLEGAL_GENERATION:
/* Revoke assignment and rebalance on illegal generation */
rk->rk_cgrp->rkcg_generation_id = -1;
rd_kafka_cgrp_revoke_all_rejoin_maybe(
rkcg, rd_true /*assignment is lost*/,
rd_true /*this consumer is initiating*/,
"OffsetCommit error: Illegal generation");
break;
case RD_KAFKA_RESP_ERR__IN_PROGRESS:
return; /* Retrying */
case RD_KAFKA_RESP_ERR_NOT_COORDINATOR:
case RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR__TRANSPORT:
/* The coordinator is not available, defer the offset commit
* to when the coordinator is back up again. */
/* Future-proofing, see timeout_scan(). */
rd_kafka_assert(NULL, err != RD_KAFKA_RESP_ERR__WAIT_COORD);
if (rd_kafka_cgrp_defer_offset_commit(rkcg, rko_orig,
rd_kafka_err2str(err)))
return;
break;
default:
break;
}
/* Call on_commit interceptors */
if (err != RD_KAFKA_RESP_ERR__NO_OFFSET &&
err != RD_KAFKA_RESP_ERR__DESTROY && offsets && offsets->cnt > 0)
rd_kafka_interceptors_on_commit(rk, offsets, err);
/* Keep track of outstanding commits */
rd_kafka_assert(NULL, rk->rk_consumer.wait_commit_cnt > 0);
rk->rk_consumer.wait_commit_cnt--;
if (err == RD_KAFKA_RESP_ERR__DESTROY) {
rd_kafka_op_destroy(rko_orig);
return; /* Handle is terminating, this op may be handled
* by the op enq()ing thread rather than the
* rdkafka main thread, it is not safe to
* continue here. */
}
/* Update the committed offsets for each partition's rktp. */
errcnt = rd_kafka_cgrp_update_committed_offsets(rkcg, err, offsets);
if (err != RD_KAFKA_RESP_ERR__DESTROY &&
!(err == RD_KAFKA_RESP_ERR__NO_OFFSET &&
rko_orig->rko_u.offset_commit.silent_empty)) {
/* Propagate commit results (success or permanent error)
* unless we're shutting down or commit was empty. */
rd_kafka_cgrp_propagate_commit_result(rkcg, rko_orig, err,
errcnt, offsets);
}
rd_kafka_op_destroy(rko_orig);
/* If the current state was waiting for commits to finish we'll try to
* transition to the next state. */
if (rk->rk_consumer.wait_commit_cnt == 0)
rd_kafka_assignment_serve(rk);
}
static size_t rd_kafka_topic_partition_has_absolute_offset(
const rd_kafka_topic_partition_t *rktpar,
void *opaque) {
return rktpar->offset >= 0 ? 1 : 0;
}
/**
* Commit a list of offsets.
* Reuse the orignating 'rko' for the async reply.
* 'rko->rko_payload' should either by NULL (to commit current assignment) or
* a proper topic_partition_list_t with offsets to commit.
* The offset list will be altered.
*
* \p rko...silent_empty: if there are no offsets to commit bail out
* silently without posting an op on the reply queue.
* \p set_offsets: set offsets and epochs in
* rko->rko_u.offset_commit.partitions from the rktp's
* stored offset.
*
* Locality: cgrp thread
*/
static void rd_kafka_cgrp_offsets_commit(rd_kafka_cgrp_t *rkcg,
rd_kafka_op_t *rko,
rd_bool_t set_offsets,
const char *reason) {
rd_kafka_topic_partition_list_t *offsets;
rd_kafka_resp_err_t err;
int valid_offsets = 0;
int r;
rd_kafka_buf_t *rkbuf;
rd_kafka_op_t *reply;
rd_kafka_consumer_group_metadata_t *cgmetadata;
if (!(rko->rko_flags & RD_KAFKA_OP_F_REPROCESS)) {
/* wait_commit_cnt has already been increased for
* reprocessed ops. */
rkcg->rkcg_rk->rk_consumer.wait_commit_cnt++;
}
/* If offsets is NULL we shall use the current assignment
* (not the group assignment). */
if (!rko->rko_u.offset_commit.partitions &&
rkcg->rkcg_rk->rk_consumer.assignment.all->cnt > 0) {
if (rd_kafka_cgrp_assignment_is_lost(rkcg)) {
/* Not committing assigned offsets: assignment lost */
err = RD_KAFKA_RESP_ERR__ASSIGNMENT_LOST;
goto err;
}
rko->rko_u.offset_commit.partitions =
rd_kafka_topic_partition_list_copy(
rkcg->rkcg_rk->rk_consumer.assignment.all);
}
offsets = rko->rko_u.offset_commit.partitions;
if (offsets) {
/* Set offsets to commits */
if (set_offsets)
rd_kafka_topic_partition_list_set_offsets(
rkcg->rkcg_rk, rko->rko_u.offset_commit.partitions,
1, RD_KAFKA_OFFSET_INVALID /* def */,
1 /* is commit */);
/* Check the number of valid offsets to commit. */
valid_offsets = (int)rd_kafka_topic_partition_list_sum(
offsets, rd_kafka_topic_partition_has_absolute_offset,
NULL);
}
if (rd_kafka_fatal_error_code(rkcg->rkcg_rk)) {
/* Commits are not allowed when a fatal error has been raised */
err = RD_KAFKA_RESP_ERR__FATAL;
goto err;
}
if (!valid_offsets) {
/* No valid offsets */
err = RD_KAFKA_RESP_ERR__NO_OFFSET;
goto err;
}
if (rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP) {
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP,
"COMMIT",
"Deferring \"%s\" offset commit "
"for %d partition(s) in state %s: "
"no coordinator available",
reason, valid_offsets,
rd_kafka_cgrp_state_names[rkcg->rkcg_state]);
if (rd_kafka_cgrp_defer_offset_commit(rkcg, rko, reason))
return;
err = RD_KAFKA_RESP_ERR__WAIT_COORD;
goto err;
}
rd_rkb_dbg(rkcg->rkcg_coord, CONSUMER | RD_KAFKA_DBG_CGRP, "COMMIT",
"Committing offsets for %d partition(s) with "
"generation-id %" PRId32 " in join-state %s: %s",
valid_offsets, rkcg->rkcg_generation_id,
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
reason);
cgmetadata = rd_kafka_consumer_group_metadata_new_with_genid(
rkcg->rkcg_rk->rk_conf.group_id_str, rkcg->rkcg_generation_id,
rkcg->rkcg_member_id->str,
rkcg->rkcg_rk->rk_conf.group_instance_id);
/* Send OffsetCommit */
r = rd_kafka_OffsetCommitRequest(rkcg->rkcg_coord, cgmetadata, offsets,
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_cgrp_op_handle_OffsetCommit,
rko, reason);
rd_kafka_consumer_group_metadata_destroy(cgmetadata);
/* Must have valid offsets to commit if we get here */
rd_kafka_assert(NULL, r != 0);
return;
err:
if (err != RD_KAFKA_RESP_ERR__NO_OFFSET)
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP,
"COMMIT", "OffsetCommit internal error: %s",
rd_kafka_err2str(err));
/* Propagate error through dummy buffer object that will
* call the response handler from the main loop, avoiding
* any recursive calls from op_handle_OffsetCommit ->
* assignment_serve() and then back to cgrp_assigned_offsets_commit() */
reply = rd_kafka_op_new(RD_KAFKA_OP_RECV_BUF);
reply->rko_rk = rkcg->rkcg_rk; /* Set rk since the rkbuf will not
* have a rkb to reach it. */
reply->rko_err = err;
rkbuf = rd_kafka_buf_new(0, 0);
rkbuf->rkbuf_cb = rd_kafka_cgrp_op_handle_OffsetCommit;
rkbuf->rkbuf_opaque = rko;
reply->rko_u.xbuf.rkbuf = rkbuf;
rd_kafka_q_enq(rkcg->rkcg_ops, reply);
}
/**
* @brief Commit offsets assigned partitions.
*
* If \p offsets is NULL all partitions in the current assignment will be used.
* If \p set_offsets is true the offsets to commit will be read from the
* rktp's stored offset rather than the .offset fields in \p offsets.
*
* rkcg_wait_commit_cnt will be increased accordingly.
*/
void rd_kafka_cgrp_assigned_offsets_commit(
rd_kafka_cgrp_t *rkcg,
const rd_kafka_topic_partition_list_t *offsets,
rd_bool_t set_offsets,
const char *reason) {
rd_kafka_op_t *rko;
if (rd_kafka_cgrp_assignment_is_lost(rkcg)) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "AUTOCOMMIT",
"Group \"%s\": not committing assigned offsets: "
"assignment lost",
rkcg->rkcg_group_id->str);
return;
}
rko = rd_kafka_op_new(RD_KAFKA_OP_OFFSET_COMMIT);
rko->rko_u.offset_commit.reason = rd_strdup(reason);
if (rkcg->rkcg_rk->rk_conf.enabled_events &
RD_KAFKA_EVENT_OFFSET_COMMIT) {
/* Send results to application */
rd_kafka_op_set_replyq(rko, rkcg->rkcg_rk->rk_rep, 0);
rko->rko_u.offset_commit.cb =
rkcg->rkcg_rk->rk_conf.offset_commit_cb; /*maybe NULL*/
rko->rko_u.offset_commit.opaque = rkcg->rkcg_rk->rk_conf.opaque;
}
/* NULL partitions means current assignment */
if (offsets)
rko->rko_u.offset_commit.partitions =
rd_kafka_topic_partition_list_copy(offsets);
rko->rko_u.offset_commit.silent_empty = 1;
rd_kafka_cgrp_offsets_commit(rkcg, rko, set_offsets, reason);
}
/**
* auto.commit.interval.ms commit timer callback.
*
* Trigger a group offset commit.
*
* Locality: rdkafka main thread
*/
static void rd_kafka_cgrp_offset_commit_tmr_cb(rd_kafka_timers_t *rkts,
void *arg) {
rd_kafka_cgrp_t *rkcg = arg;
/* Don't attempt auto commit when rebalancing or initializing since
* the rkcg_generation_id is most likely in flux. */
if (rkcg->rkcg_subscription &&
rkcg->rkcg_join_state != RD_KAFKA_CGRP_JOIN_STATE_STEADY)
return;
rd_kafka_cgrp_assigned_offsets_commit(
rkcg, NULL, rd_true /*set offsets*/, "cgrp auto commit timer");
}
/**
* @brief If rkcg_next_subscription or rkcg_next_unsubscribe are
* set, trigger a state change so that they are applied from the
* main dispatcher.
*
* @returns rd_true if a subscribe was scheduled, else false.
*/
static rd_bool_t
rd_kafka_trigger_waiting_subscribe_maybe(rd_kafka_cgrp_t *rkcg) {
if (rkcg->rkcg_next_subscription || rkcg->rkcg_next_unsubscribe) {
/* Skip the join backoff */
rd_interval_reset(&rkcg->rkcg_join_intvl);
rd_kafka_cgrp_rejoin(rkcg, "Applying next subscription");
return rd_true;
}
return rd_false;
}
/**
* @brief Incrementally add to an existing partition assignment
* May update \p partitions but will not hold on to it.
*
* @returns an error object or NULL on success.
*/
static rd_kafka_error_t *
rd_kafka_cgrp_incremental_assign(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *partitions) {
rd_kafka_error_t *error;
error = rd_kafka_assignment_add(rkcg->rkcg_rk, partitions);
if (error)
return error;
if (rkcg->rkcg_join_state ==
RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL) {
rd_kafka_assignment_resume(rkcg->rkcg_rk,
"incremental assign called");
rd_kafka_cgrp_set_join_state(rkcg,
RD_KAFKA_CGRP_JOIN_STATE_STEADY);
if (rkcg->rkcg_subscription) {
/* If using subscribe(), start a timer to enforce
* `max.poll.interval.ms`.
* Instead of restarting the timer on each ...poll()
* call, which would be costly (once per message),
* set up an intervalled timer that checks a timestamp
* (that is updated on ..poll()).
* The timer interval is 2 hz. */
rd_kafka_timer_start(
&rkcg->rkcg_rk->rk_timers,
&rkcg->rkcg_max_poll_interval_tmr,
500 * 1000ll /* 500ms */,
rd_kafka_cgrp_max_poll_interval_check_tmr_cb, rkcg);
}
}
rd_kafka_cgrp_assignment_clear_lost(rkcg,
"incremental_assign() called");
return NULL;
}
/**
* @brief Incrementally remove partitions from an existing partition
* assignment. May update \p partitions but will not hold on
* to it.
*
* @remark This method does not unmark the current assignment as lost
* (if lost). That happens following _incr_unassign_done and
* a group-rejoin initiated.
*
* @returns An error object or NULL on success.
*/
static rd_kafka_error_t *rd_kafka_cgrp_incremental_unassign(
rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *partitions) {
rd_kafka_error_t *error;
error = rd_kafka_assignment_subtract(rkcg->rkcg_rk, partitions);
if (error)
return error;
if (rkcg->rkcg_join_state ==
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL) {
rd_kafka_assignment_resume(rkcg->rkcg_rk,
"incremental unassign called");
rd_kafka_cgrp_set_join_state(
rkcg,
RD_KAFKA_CGRP_JOIN_STATE_WAIT_INCR_UNASSIGN_TO_COMPLETE);
}
rd_kafka_cgrp_assignment_clear_lost(rkcg,
"incremental_unassign() called");
return NULL;
}
/**
* @brief Call when all incremental unassign operations are done to transition
* to the next state.
*/
static void rd_kafka_cgrp_incr_unassign_done(rd_kafka_cgrp_t *rkcg) {
/* If this action was underway when a terminate was initiated, it will
* be left to complete. Now that's done, unassign all partitions */
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "UNASSIGN",
"Group \"%s\" is terminating, initiating full "
"unassign",
rkcg->rkcg_group_id->str);
rd_kafka_cgrp_unassign(rkcg);
return;
}
if (rkcg->rkcg_rebalance_incr_assignment) {
/* This incremental unassign was part of a normal rebalance
* (in which the revoke set was not empty). Immediately
* trigger the assign that follows this revoke. The protocol
* dictates this should occur even if the new assignment
* set is empty.
*
* Also, since this rebalance had some revoked partitions,
* a re-join should occur following the assign.
*/
rd_kafka_rebalance_op_incr(rkcg,
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS,
rkcg->rkcg_rebalance_incr_assignment,
rd_true /*rejoin following assign*/,
"cooperative assign after revoke");
rd_kafka_topic_partition_list_destroy(
rkcg->rkcg_rebalance_incr_assignment);
rkcg->rkcg_rebalance_incr_assignment = NULL;
/* Note: rkcg_rebalance_rejoin is actioned / reset in
* rd_kafka_cgrp_incremental_assign call */
} else if (rkcg->rkcg_rebalance_rejoin) {
rkcg->rkcg_rebalance_rejoin = rd_false;
/* There are some cases (lost partitions), where a rejoin
* should occur immediately following the unassign (this
* is not the case under normal conditions), in which case
* the rejoin flag will be set. */
/* Skip the join backoff */
rd_interval_reset(&rkcg->rkcg_join_intvl);
rd_kafka_cgrp_rejoin(rkcg, "Incremental unassignment done");
} else if (!rd_kafka_trigger_waiting_subscribe_maybe(rkcg)) {
/* After this incremental unassignment we're now back in
* a steady state. */
rd_kafka_cgrp_set_join_state(rkcg,
RD_KAFKA_CGRP_JOIN_STATE_STEADY);
}
}
/**
* @brief Call when all absolute (non-incremental) unassign operations are done
* to transition to the next state.
*/
static void rd_kafka_cgrp_unassign_done(rd_kafka_cgrp_t *rkcg) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "UNASSIGN",
"Group \"%s\": unassign done in state %s "
"(join-state %s)",
rkcg->rkcg_group_id->str,
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
/* Leave group, if desired. */
rd_kafka_cgrp_leave_maybe(rkcg);
if (rkcg->rkcg_join_state !=
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_TO_COMPLETE)
return;
/* All partitions are unassigned. Rejoin the group. */
/* Skip the join backoff */
rd_interval_reset(&rkcg->rkcg_join_intvl);
rd_kafka_cgrp_rejoin(rkcg, "Unassignment done");
}
/**
* @brief Called from assignment code when all in progress
* assignment/unassignment operations are done, allowing the cgrp to
* transition to other states if needed.
*
* @remark This may be called spontaneously without any need for a state
* change in the rkcg.
*/
void rd_kafka_cgrp_assignment_done(rd_kafka_cgrp_t *rkcg) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "ASSIGNDONE",
"Group \"%s\": "
"assignment operations done in join-state %s "
"(rebalance rejoin=%s)",
rkcg->rkcg_group_id->str,
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
RD_STR_ToF(rkcg->rkcg_rebalance_rejoin));
switch (rkcg->rkcg_join_state) {
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_TO_COMPLETE:
rd_kafka_cgrp_unassign_done(rkcg);
break;
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_INCR_UNASSIGN_TO_COMPLETE:
rd_kafka_cgrp_incr_unassign_done(rkcg);
break;
case RD_KAFKA_CGRP_JOIN_STATE_STEADY:
/* If an updated/next subscription is available, schedule it. */
if (rd_kafka_trigger_waiting_subscribe_maybe(rkcg))
break;
if (rkcg->rkcg_rebalance_rejoin) {
rkcg->rkcg_rebalance_rejoin = rd_false;
/* Skip the join backoff */
rd_interval_reset(&rkcg->rkcg_join_intvl);
rd_kafka_cgrp_rejoin(
rkcg,
"rejoining group to redistribute "
"previously owned partitions to other "
"group members");
break;
}
/* FALLTHRU */
case RD_KAFKA_CGRP_JOIN_STATE_INIT:
/* Check if cgrp is trying to terminate, which is safe to do
* in these two states. Otherwise we'll need to wait for
* the current state to decommission. */
rd_kafka_cgrp_try_terminate(rkcg);
break;
default:
break;
}
}
/**
* @brief Remove existing assignment.
*/
static rd_kafka_error_t *rd_kafka_cgrp_unassign(rd_kafka_cgrp_t *rkcg) {
rd_kafka_assignment_clear(rkcg->rkcg_rk);
if (rkcg->rkcg_join_state ==
RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL) {
rd_kafka_assignment_resume(rkcg->rkcg_rk, "unassign called");
rd_kafka_cgrp_set_join_state(
rkcg, RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_TO_COMPLETE);
}
rd_kafka_cgrp_assignment_clear_lost(rkcg, "unassign() called");
return NULL;
}
/**
* @brief Set new atomic partition assignment
* May update \p assignment but will not hold on to it.
*
* @returns NULL on success or an error if a fatal error has been raised.
*/
static rd_kafka_error_t *
rd_kafka_cgrp_assign(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *assignment) {
rd_kafka_error_t *error;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER, "ASSIGN",
"Group \"%s\": new assignment of %d partition(s) "
"in join-state %s",
rkcg->rkcg_group_id->str, assignment ? assignment->cnt : 0,
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
/* Clear existing assignment, if any, and serve its removals. */
if (rd_kafka_assignment_clear(rkcg->rkcg_rk))
rd_kafka_assignment_serve(rkcg->rkcg_rk);
error = rd_kafka_assignment_add(rkcg->rkcg_rk, assignment);
if (error)
return error;
rd_kafka_cgrp_assignment_clear_lost(rkcg, "assign() called");
if (rkcg->rkcg_join_state ==
RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL) {
rd_kafka_assignment_resume(rkcg->rkcg_rk, "assign called");
rd_kafka_cgrp_set_join_state(rkcg,
RD_KAFKA_CGRP_JOIN_STATE_STEADY);
if (rkcg->rkcg_subscription) {
/* If using subscribe(), start a timer to enforce
* `max.poll.interval.ms`.
* Instead of restarting the timer on each ...poll()
* call, which would be costly (once per message),
* set up an intervalled timer that checks a timestamp
* (that is updated on ..poll()).
* The timer interval is 2 hz. */
rd_kafka_timer_start(
&rkcg->rkcg_rk->rk_timers,
&rkcg->rkcg_max_poll_interval_tmr,
500 * 1000ll /* 500ms */,
rd_kafka_cgrp_max_poll_interval_check_tmr_cb, rkcg);
}
}
return NULL;
}
/**
* @brief Construct a typed map from list \p rktparlist with key corresponding
* to each element in the list and value NULL.
*
* @remark \p rktparlist may be NULL.
*/
static map_toppar_member_info_t *rd_kafka_toppar_list_to_toppar_member_info_map(
rd_kafka_topic_partition_list_t *rktparlist) {
map_toppar_member_info_t *map = rd_calloc(1, sizeof(*map));
const rd_kafka_topic_partition_t *rktpar;
RD_MAP_INIT(map, rktparlist ? rktparlist->cnt : 0,
rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free,
PartitionMemberInfo_free);
if (!rktparlist)
return map;
RD_KAFKA_TPLIST_FOREACH(rktpar, rktparlist)
RD_MAP_SET(map, rd_kafka_topic_partition_copy(rktpar),
PartitionMemberInfo_new(NULL, rd_false));
return map;
}
/**
* @brief Construct a toppar list from map \p map with elements corresponding
* to the keys of \p map.
*/
static rd_kafka_topic_partition_list_t *
rd_kafka_toppar_member_info_map_to_list(map_toppar_member_info_t *map) {
const rd_kafka_topic_partition_t *k;
rd_kafka_topic_partition_list_t *list =
rd_kafka_topic_partition_list_new((int)RD_MAP_CNT(map));
RD_MAP_FOREACH_KEY(k, map) {
rd_kafka_topic_partition_list_add(list, k->topic, k->partition);
}
return list;
}
/**
* @brief Handle a rebalance-triggered partition assignment
* (COOPERATIVE case).
*/
static void rd_kafka_cgrp_handle_assignment_cooperative(
rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *assignment) {
map_toppar_member_info_t *new_assignment_set;
map_toppar_member_info_t *old_assignment_set;
map_toppar_member_info_t *newly_added_set;
map_toppar_member_info_t *revoked_set;
rd_kafka_topic_partition_list_t *newly_added;
rd_kafka_topic_partition_list_t *revoked;
new_assignment_set =
rd_kafka_toppar_list_to_toppar_member_info_map(assignment);
old_assignment_set = rd_kafka_toppar_list_to_toppar_member_info_map(
rkcg->rkcg_group_assignment);
newly_added_set = rd_kafka_member_partitions_subtract(
new_assignment_set, old_assignment_set);
revoked_set = rd_kafka_member_partitions_subtract(old_assignment_set,
new_assignment_set);
newly_added = rd_kafka_toppar_member_info_map_to_list(newly_added_set);
revoked = rd_kafka_toppar_member_info_map_to_list(revoked_set);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COOPASSIGN",
"Group \"%s\": incremental assignment: %d newly added, "
"%d revoked partitions based on assignment of %d "
"partitions",
rkcg->rkcg_group_id->str, newly_added->cnt, revoked->cnt,
assignment->cnt);
if (revoked->cnt > 0) {
/* Setting rkcg_incr_assignment causes a follow on incremental
* assign rebalance op after completion of this incremental
* unassign op. */
rkcg->rkcg_rebalance_incr_assignment = newly_added;
newly_added = NULL;
rd_kafka_rebalance_op_incr(rkcg,
RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS,
revoked, rd_false /*no rejoin
following unassign*/
,
"sync group revoke");
} else {
/* There are no revoked partitions - trigger the assign
* rebalance op, and flag that the group does not need
* to be re-joined */
rd_kafka_rebalance_op_incr(
rkcg, RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS, newly_added,
rd_false /*no rejoin following assign*/,
"sync group assign");
}
if (newly_added)
rd_kafka_topic_partition_list_destroy(newly_added);
rd_kafka_topic_partition_list_destroy(revoked);
RD_MAP_DESTROY_AND_FREE(revoked_set);
RD_MAP_DESTROY_AND_FREE(newly_added_set);
RD_MAP_DESTROY_AND_FREE(old_assignment_set);
RD_MAP_DESTROY_AND_FREE(new_assignment_set);
}
/**
* @brief Sets or clears the group's partition assignment for our consumer.
*
* Will replace the current group assignment, if any.
*/
static void rd_kafka_cgrp_group_assignment_set(
rd_kafka_cgrp_t *rkcg,
const rd_kafka_topic_partition_list_t *partitions) {
if (rkcg->rkcg_group_assignment)
rd_kafka_topic_partition_list_destroy(
rkcg->rkcg_group_assignment);
if (partitions) {
rkcg->rkcg_group_assignment =
rd_kafka_topic_partition_list_copy(partitions);
rd_kafka_topic_partition_list_sort_by_topic(
rkcg->rkcg_group_assignment);
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "ASSIGNMENT",
"Group \"%s\": setting group assignment to %d "
"partition(s)",
rkcg->rkcg_group_id->str,
rkcg->rkcg_group_assignment->cnt);
} else {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "ASSIGNMENT",
"Group \"%s\": clearing group assignment",
rkcg->rkcg_group_id->str);
rkcg->rkcg_group_assignment = NULL;
}
rd_kafka_wrlock(rkcg->rkcg_rk);
rkcg->rkcg_c.assignment_size =
rkcg->rkcg_group_assignment ? rkcg->rkcg_group_assignment->cnt : 0;
rd_kafka_wrunlock(rkcg->rkcg_rk);
if (rkcg->rkcg_group_assignment)
rd_kafka_topic_partition_list_log(
rkcg->rkcg_rk, "GRPASSIGNMENT", RD_KAFKA_DBG_CGRP,
rkcg->rkcg_group_assignment);
}
/**
* @brief Adds or removes \p partitions from the current group assignment.
*
* @param add Whether to add or remove the partitions.
*
* @remark The added partitions must not already be on the group assignment,
* and the removed partitions must be on the group assignment.
*
* To be used with incremental rebalancing.
*
*/
static void rd_kafka_cgrp_group_assignment_modify(
rd_kafka_cgrp_t *rkcg,
rd_bool_t add,
const rd_kafka_topic_partition_list_t *partitions) {
const rd_kafka_topic_partition_t *rktpar;
int precnt;
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP, "ASSIGNMENT",
"Group \"%s\": %d partition(s) being %s group assignment "
"of %d partition(s)",
rkcg->rkcg_group_id->str, partitions->cnt,
add ? "added to" : "removed from",
rkcg->rkcg_group_assignment ? rkcg->rkcg_group_assignment->cnt : 0);
if (partitions == rkcg->rkcg_group_assignment) {
/* \p partitions is the actual assignment, which
* must mean it is all to be removed.
* Short-cut directly to set(NULL). */
rd_assert(!add);
rd_kafka_cgrp_group_assignment_set(rkcg, NULL);
return;
}
if (add && (!rkcg->rkcg_group_assignment ||
rkcg->rkcg_group_assignment->cnt == 0)) {
/* Adding to an empty assignment is a set operation. */
rd_kafka_cgrp_group_assignment_set(rkcg, partitions);
return;
}
if (!add) {
/* Removing from an empty assignment is illegal. */
rd_assert(rkcg->rkcg_group_assignment != NULL &&
rkcg->rkcg_group_assignment->cnt > 0);
}
precnt = rkcg->rkcg_group_assignment->cnt;
RD_KAFKA_TPLIST_FOREACH(rktpar, partitions) {
int idx;
idx = rd_kafka_topic_partition_list_find_idx(
rkcg->rkcg_group_assignment, rktpar->topic,
rktpar->partition);
if (add) {
rd_assert(idx == -1);
rd_kafka_topic_partition_list_add_copy(
rkcg->rkcg_group_assignment, rktpar);
} else {
rd_assert(idx != -1);
rd_kafka_topic_partition_list_del_by_idx(
rkcg->rkcg_group_assignment, idx);
}
}
if (add)
rd_assert(precnt + partitions->cnt ==
rkcg->rkcg_group_assignment->cnt);
else
rd_assert(precnt - partitions->cnt ==
rkcg->rkcg_group_assignment->cnt);
if (rkcg->rkcg_group_assignment->cnt == 0) {
rd_kafka_topic_partition_list_destroy(
rkcg->rkcg_group_assignment);
rkcg->rkcg_group_assignment = NULL;
} else if (add)
rd_kafka_topic_partition_list_sort_by_topic(
rkcg->rkcg_group_assignment);
rd_kafka_wrlock(rkcg->rkcg_rk);
rkcg->rkcg_c.assignment_size =
rkcg->rkcg_group_assignment ? rkcg->rkcg_group_assignment->cnt : 0;
rd_kafka_wrunlock(rkcg->rkcg_rk);
if (rkcg->rkcg_group_assignment)
rd_kafka_topic_partition_list_log(
rkcg->rkcg_rk, "GRPASSIGNMENT", RD_KAFKA_DBG_CGRP,
rkcg->rkcg_group_assignment);
}
/**
* @brief Handle a rebalance-triggered partition assignment.
*
* If a rebalance_cb has been registered we enqueue an op for the app
* and let the app perform the actual assign() call. Otherwise we
* assign() directly from here.
*
* This provides the most flexibility, allowing the app to perform any
* operation it seem fit (e.g., offset writes or reads) before actually
* updating the assign():ment.
*/
static void
rd_kafka_cgrp_handle_assignment(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *assignment) {
if (rd_kafka_cgrp_rebalance_protocol(rkcg) ==
RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE) {
rd_kafka_cgrp_handle_assignment_cooperative(rkcg, assignment);
} else {
rd_kafka_rebalance_op(rkcg,
RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS,
assignment, "new assignment");
}
}
/**
* Clean up any group-leader related resources.
*
* Locality: cgrp thread
*/
static void rd_kafka_cgrp_group_leader_reset(rd_kafka_cgrp_t *rkcg,
const char *reason) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "GRPLEADER",
"Group \"%.*s\": resetting group leader info: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), reason);
if (rkcg->rkcg_group_leader.members) {
int i;
for (i = 0; i < rkcg->rkcg_group_leader.member_cnt; i++)
rd_kafka_group_member_clear(
&rkcg->rkcg_group_leader.members[i]);
rkcg->rkcg_group_leader.member_cnt = 0;
rd_free(rkcg->rkcg_group_leader.members);
rkcg->rkcg_group_leader.members = NULL;
}
}
/**
* @brief React to a RD_KAFKA_RESP_ERR_REBALANCE_IN_PROGRESS broker response.
*/
static void rd_kafka_cgrp_group_is_rebalancing(rd_kafka_cgrp_t *rkcg) {
if (rd_kafka_cgrp_rebalance_protocol(rkcg) ==
RD_KAFKA_REBALANCE_PROTOCOL_EAGER) {
rd_kafka_cgrp_revoke_all_rejoin_maybe(rkcg, rd_false /*lost*/,
rd_false /*initiating*/,
"rebalance in progress");
return;
}
/* In the COOPERATIVE case, simply rejoin the group
* - partitions are unassigned on SyncGroup response,
* not prior to JoinGroup as with the EAGER case. */
if (RD_KAFKA_CGRP_REBALANCING(rkcg)) {
rd_kafka_dbg(
rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "REBALANCE",
"Group \"%.*s\": skipping "
"COOPERATIVE rebalance in state %s "
"(join-state %s)%s%s%s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
RD_KAFKA_CGRP_WAIT_ASSIGN_CALL(rkcg)
? " (awaiting assign call)"
: "",
(rkcg->rkcg_rebalance_incr_assignment != NULL)
? " (incremental assignment pending)"
: "",
rkcg->rkcg_rebalance_rejoin ? " (rebalance rejoin)" : "");
return;
}
rd_kafka_cgrp_rejoin(rkcg, "Group is rebalancing");
}
/**
* @brief Triggers the application rebalance callback if required to
* revoke partitions, and transition to INIT state for (eventual)
* rejoin. Does nothing if a rebalance workflow is already in
* progress
*/
static void rd_kafka_cgrp_revoke_all_rejoin_maybe(rd_kafka_cgrp_t *rkcg,
rd_bool_t assignment_lost,
rd_bool_t initiating,
const char *reason) {
if (RD_KAFKA_CGRP_REBALANCING(rkcg)) {
rd_kafka_dbg(
rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "REBALANCE",
"Group \"%.*s\": rebalance (%s) "
"already in progress, skipping in state %s "
"(join-state %s) with %d assigned partition(s)%s%s%s: "
"%s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_rebalance_protocol2str(
rd_kafka_cgrp_rebalance_protocol(rkcg)),
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
rkcg->rkcg_group_assignment
? rkcg->rkcg_group_assignment->cnt
: 0,
assignment_lost ? " (lost)" : "",
rkcg->rkcg_rebalance_incr_assignment
? ", incremental assignment in progress"
: "",
rkcg->rkcg_rebalance_rejoin ? ", rejoin on rebalance" : "",
reason);
return;
}
rd_kafka_cgrp_revoke_all_rejoin(rkcg, assignment_lost, initiating,
reason);
}
/**
* @brief Triggers the application rebalance callback if required to
* revoke partitions, and transition to INIT state for (eventual)
* rejoin.
*/
static void rd_kafka_cgrp_revoke_all_rejoin(rd_kafka_cgrp_t *rkcg,
rd_bool_t assignment_lost,
rd_bool_t initiating,
const char *reason) {
rd_kafka_rebalance_protocol_t protocol =
rd_kafka_cgrp_rebalance_protocol(rkcg);
rd_bool_t terminating =
unlikely(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE);
rd_kafka_dbg(
rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP, "REBALANCE",
"Group \"%.*s\" %s (%s) in state %s (join-state %s) "
"with %d assigned partition(s)%s: %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
initiating ? "initiating rebalance" : "is rebalancing",
rd_kafka_rebalance_protocol2str(protocol),
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
rkcg->rkcg_group_assignment ? rkcg->rkcg_group_assignment->cnt : 0,
assignment_lost ? " (lost)" : "", reason);
rd_snprintf(rkcg->rkcg_c.rebalance_reason,
sizeof(rkcg->rkcg_c.rebalance_reason), "%s", reason);
if (protocol == RD_KAFKA_REBALANCE_PROTOCOL_EAGER ||
protocol == RD_KAFKA_REBALANCE_PROTOCOL_NONE) {
/* EAGER case (or initial subscribe) - revoke partitions which
* will be followed by rejoin, if required. */
if (assignment_lost)
rd_kafka_cgrp_assignment_set_lost(
rkcg, "%s: revoking assignment and rejoining",
reason);
/* Schedule application rebalance op if there is an existing
* assignment (albeit perhaps empty) and there is no
* outstanding rebalance op in progress. */
if (rkcg->rkcg_group_assignment &&
!RD_KAFKA_CGRP_WAIT_ASSIGN_CALL(rkcg)) {
rd_kafka_rebalance_op(
rkcg, RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS,
rkcg->rkcg_group_assignment, reason);
} else {
/* Skip the join backoff */
rd_interval_reset(&rkcg->rkcg_join_intvl);
rd_kafka_cgrp_rejoin(rkcg, "%s", reason);
}
return;
}
/* COOPERATIVE case. */
/* All partitions should never be revoked unless terminating, leaving
* the group, or on assignment lost. Another scenario represents a
* logic error. Fail fast in this case. */
if (!(terminating || assignment_lost ||
(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE))) {
rd_kafka_log(rkcg->rkcg_rk, LOG_ERR, "REBALANCE",
"Group \"%s\": unexpected instruction to revoke "
"current assignment and rebalance "
"(terminating=%d, assignment_lost=%d, "
"LEAVE_ON_UNASSIGN_DONE=%d)",
rkcg->rkcg_group_id->str, terminating,
assignment_lost,
(rkcg->rkcg_flags &
RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE));
rd_dassert(!*"BUG: unexpected instruction to revoke "
"current assignment and rebalance");
}
if (rkcg->rkcg_group_assignment &&
rkcg->rkcg_group_assignment->cnt > 0) {
if (assignment_lost)
rd_kafka_cgrp_assignment_set_lost(
rkcg,
"%s: revoking incremental assignment "
"and rejoining",
reason);
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP,
"REBALANCE",
"Group \"%.*s\": revoking "
"all %d partition(s)%s%s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rkcg->rkcg_group_assignment->cnt,
terminating ? " (terminating)" : "",
assignment_lost ? " (assignment lost)" : "");
rd_kafka_rebalance_op_incr(
rkcg, RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS,
rkcg->rkcg_group_assignment,
terminating ? rd_false : rd_true /*rejoin*/, reason);
return;
}
if (terminating) {
/* If terminating, then don't rejoin group. */
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP,
"REBALANCE",
"Group \"%.*s\": consumer is "
"terminating, skipping rejoin",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id));
return;
}
rd_kafka_cgrp_rejoin(rkcg, "Current assignment is empty");
}
/**
* @brief `max.poll.interval.ms` enforcement check timer.
*
* @locality rdkafka main thread
* @locks none
*/
static void
rd_kafka_cgrp_max_poll_interval_check_tmr_cb(rd_kafka_timers_t *rkts,
void *arg) {
rd_kafka_cgrp_t *rkcg = arg;
rd_kafka_t *rk = rkcg->rkcg_rk;
int exceeded;
exceeded = rd_kafka_max_poll_exceeded(rk);
if (likely(!exceeded))
return;
rd_kafka_log(rk, LOG_WARNING, "MAXPOLL",
"Application maximum poll interval (%dms) "
"exceeded by %dms "
"(adjust max.poll.interval.ms for "
"long-running message processing): "
"leaving group",
rk->rk_conf.max_poll_interval_ms, exceeded);
rd_kafka_consumer_err(rkcg->rkcg_q, RD_KAFKA_NODEID_UA,
RD_KAFKA_RESP_ERR__MAX_POLL_EXCEEDED, 0, NULL,
NULL, RD_KAFKA_OFFSET_INVALID,
"Application maximum poll interval (%dms) "
"exceeded by %dms",
rk->rk_conf.max_poll_interval_ms, exceeded);
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_MAX_POLL_EXCEEDED;
rd_kafka_timer_stop(rkts, &rkcg->rkcg_max_poll_interval_tmr,
1 /*lock*/);
/* Leave the group before calling rebalance since the standard leave
* will be triggered first after the rebalance callback has been served.
* But since the application is blocked still doing processing
* that leave will be further delayed.
*
* KIP-345: static group members should continue to respect
* `max.poll.interval.ms` but should not send a LeaveGroupRequest.
*/
if (!RD_KAFKA_CGRP_IS_STATIC_MEMBER(rkcg))
rd_kafka_cgrp_leave(rkcg);
/* Timing out or leaving the group invalidates the member id, reset it
* now to avoid an ERR_UNKNOWN_MEMBER_ID on the next join. */
rd_kafka_cgrp_set_member_id(rkcg, "");
/* Trigger rebalance */
rd_kafka_cgrp_revoke_all_rejoin_maybe(rkcg, rd_true /*lost*/,
rd_true /*initiating*/,
"max.poll.interval.ms exceeded");
}
/**
* @brief Generate consumer errors for each topic in the list.
*
* Also replaces the list of last reported topic errors so that repeated
* errors are silenced.
*
* @param errored Errored topics.
* @param error_prefix Error message prefix.
*
* @remark Assumes ownership of \p errored.
*/
static void rd_kafka_propagate_consumer_topic_errors(
rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *errored,
const char *error_prefix) {
int i;
for (i = 0; i < errored->cnt; i++) {
rd_kafka_topic_partition_t *topic = &errored->elems[i];
rd_kafka_topic_partition_t *prev;
rd_assert(topic->err);
/* Normalize error codes, unknown topic may be
* reported by the broker, or the lack of a topic in
* metadata response is figured out by the client.
* Make sure the application only sees one error code
* for both these cases. */
if (topic->err == RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC)
topic->err = RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART;
/* Check if this topic errored previously */
prev = rd_kafka_topic_partition_list_find(
rkcg->rkcg_errored_topics, topic->topic,
RD_KAFKA_PARTITION_UA);
if (prev && prev->err == topic->err)
continue; /* This topic already reported same error */
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_TOPIC,
"TOPICERR", "%s: %s: %s", error_prefix,
topic->topic, rd_kafka_err2str(topic->err));
/* Send consumer error to application */
rd_kafka_consumer_err(
rkcg->rkcg_q, RD_KAFKA_NODEID_UA, topic->err, 0,
topic->topic, NULL, RD_KAFKA_OFFSET_INVALID, "%s: %s: %s",
error_prefix, topic->topic, rd_kafka_err2str(topic->err));
}
rd_kafka_topic_partition_list_destroy(rkcg->rkcg_errored_topics);
rkcg->rkcg_errored_topics = errored;
}
/**
* @brief Work out the topics currently subscribed to that do not
* match any pattern in \p subscription.
*/
static rd_kafka_topic_partition_list_t *rd_kafka_cgrp_get_unsubscribing_topics(
rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *subscription) {
int i;
rd_kafka_topic_partition_list_t *result;
result = rd_kafka_topic_partition_list_new(
rkcg->rkcg_subscribed_topics->rl_cnt);
/* TODO: Something that isn't O(N*M) */
for (i = 0; i < rkcg->rkcg_subscribed_topics->rl_cnt; i++) {
int j;
const char *topic =
((rd_kafka_topic_info_t *)
rkcg->rkcg_subscribed_topics->rl_elems[i])
->topic;
for (j = 0; j < subscription->cnt; j++) {
const char *pattern = subscription->elems[j].topic;
if (rd_kafka_topic_match(rkcg->rkcg_rk, pattern,
topic)) {
break;
}
}
if (j == subscription->cnt)
rd_kafka_topic_partition_list_add(
result, topic, RD_KAFKA_PARTITION_UA);
}
if (result->cnt == 0) {
rd_kafka_topic_partition_list_destroy(result);
return NULL;
}
return result;
}
/**
* @brief Determine the partitions to revoke, given the topics being
* unassigned.
*/
static rd_kafka_topic_partition_list_t *
rd_kafka_cgrp_calculate_subscribe_revoking_partitions(
rd_kafka_cgrp_t *rkcg,
const rd_kafka_topic_partition_list_t *unsubscribing) {
rd_kafka_topic_partition_list_t *revoking;
const rd_kafka_topic_partition_t *rktpar;
if (!unsubscribing)
return NULL;
if (!rkcg->rkcg_group_assignment ||
rkcg->rkcg_group_assignment->cnt == 0)
return NULL;
revoking =
rd_kafka_topic_partition_list_new(rkcg->rkcg_group_assignment->cnt);
/* TODO: Something that isn't O(N*M). */
RD_KAFKA_TPLIST_FOREACH(rktpar, unsubscribing) {
const rd_kafka_topic_partition_t *assigned;
RD_KAFKA_TPLIST_FOREACH(assigned, rkcg->rkcg_group_assignment) {
if (!strcmp(assigned->topic, rktpar->topic)) {
rd_kafka_topic_partition_list_add(
revoking, assigned->topic,
assigned->partition);
continue;
}
}
}
if (revoking->cnt == 0) {
rd_kafka_topic_partition_list_destroy(revoking);
revoking = NULL;
}
return revoking;
}
/**
* @brief Handle a new subscription that is modifying an existing subscription
* in the COOPERATIVE case.
*
* @remark Assumes ownership of \p rktparlist.
*/
static rd_kafka_resp_err_t
rd_kafka_cgrp_modify_subscription(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *rktparlist) {
rd_kafka_topic_partition_list_t *unsubscribing_topics;
rd_kafka_topic_partition_list_t *revoking;
rd_list_t *tinfos;
rd_kafka_topic_partition_list_t *errored;
int metadata_age;
int old_cnt = rkcg->rkcg_subscription->cnt;
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
if (rd_kafka_topic_partition_list_regex_cnt(rktparlist) > 0)
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
/* Topics in rkcg_subscribed_topics that don't match any pattern in
the new subscription. */
unsubscribing_topics =
rd_kafka_cgrp_get_unsubscribing_topics(rkcg, rktparlist);
/* Currently assigned topic partitions that are no longer desired. */
revoking = rd_kafka_cgrp_calculate_subscribe_revoking_partitions(
rkcg, unsubscribing_topics);
rd_kafka_topic_partition_list_destroy(rkcg->rkcg_subscription);
rkcg->rkcg_subscription = rktparlist;
if (rd_kafka_cgrp_metadata_refresh(rkcg, &metadata_age,
"modify subscription") == 1) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER,
"MODSUB",
"Group \"%.*s\": postponing join until "
"up-to-date metadata is available",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id));
rd_assert(
rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT ||
/* Possible via rd_kafka_cgrp_modify_subscription */
rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY);
rd_kafka_cgrp_set_join_state(
rkcg, RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA);
/* Revoke/join will occur after metadata refresh completes */
if (revoking)
rd_kafka_topic_partition_list_destroy(revoking);
if (unsubscribing_topics)
rd_kafka_topic_partition_list_destroy(
unsubscribing_topics);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER, "SUBSCRIBE",
"Group \"%.*s\": modifying subscription of size %d to "
"new subscription of size %d, removing %d topic(s), "
"revoking %d partition(s) (join-state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), old_cnt,
rkcg->rkcg_subscription->cnt,
unsubscribing_topics ? unsubscribing_topics->cnt : 0,
revoking ? revoking->cnt : 0,
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
if (unsubscribing_topics)
rd_kafka_topic_partition_list_destroy(unsubscribing_topics);
/* Create a list of the topics in metadata that matches the new
* subscription */
tinfos = rd_list_new(rkcg->rkcg_subscription->cnt,
(void *)rd_kafka_topic_info_destroy);
/* Unmatched topics will be added to the errored list. */
errored = rd_kafka_topic_partition_list_new(0);
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION)
rd_kafka_metadata_topic_match(rkcg->rkcg_rk, tinfos,
rkcg->rkcg_subscription, errored);
else
rd_kafka_metadata_topic_filter(
rkcg->rkcg_rk, tinfos, rkcg->rkcg_subscription, errored);
/* Propagate consumer errors for any non-existent or errored topics.
* The function takes ownership of errored. */
rd_kafka_propagate_consumer_topic_errors(
rkcg, errored, "Subscribed topic not available");
if (rd_kafka_cgrp_update_subscribed_topics(rkcg, tinfos) && !revoking) {
rd_kafka_cgrp_rejoin(rkcg, "Subscription modified");
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
if (revoking) {
rd_kafka_dbg(rkcg->rkcg_rk, CONSUMER | RD_KAFKA_DBG_CGRP,
"REBALANCE",
"Group \"%.*s\" revoking "
"%d of %d partition(s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
revoking->cnt, rkcg->rkcg_group_assignment->cnt);
rd_kafka_rebalance_op_incr(
rkcg, RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS, revoking,
rd_true /*rejoin*/, "subscribe");
rd_kafka_topic_partition_list_destroy(revoking);
}
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* Remove existing topic subscription.
*/
static rd_kafka_resp_err_t rd_kafka_cgrp_unsubscribe(rd_kafka_cgrp_t *rkcg,
rd_bool_t leave_group) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "UNSUBSCRIBE",
"Group \"%.*s\": unsubscribe from current %ssubscription "
"of size %d (leave group=%s, has joined=%s, %s, "
"join-state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rkcg->rkcg_subscription ? "" : "unset ",
rkcg->rkcg_subscription ? rkcg->rkcg_subscription->cnt : 0,
RD_STR_ToF(leave_group),
RD_STR_ToF(RD_KAFKA_CGRP_HAS_JOINED(rkcg)),
rkcg->rkcg_member_id ? rkcg->rkcg_member_id->str : "n/a",
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
rd_kafka_timer_stop(&rkcg->rkcg_rk->rk_timers,
&rkcg->rkcg_max_poll_interval_tmr, 1 /*lock*/);
if (rkcg->rkcg_subscription) {
rd_kafka_topic_partition_list_destroy(rkcg->rkcg_subscription);
rkcg->rkcg_subscription = NULL;
}
rd_kafka_cgrp_update_subscribed_topics(rkcg, NULL);
/*
* Clean-up group leader duties, if any.
*/
rd_kafka_cgrp_group_leader_reset(rkcg, "unsubscribe");
if (leave_group && RD_KAFKA_CGRP_HAS_JOINED(rkcg))
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_LEAVE_ON_UNASSIGN_DONE;
/* FIXME: Why are we only revoking if !assignment_lost ? */
if (!rd_kafka_cgrp_assignment_is_lost(rkcg))
rd_kafka_cgrp_revoke_all_rejoin(rkcg, rd_false /*not lost*/,
rd_true /*initiating*/,
"unsubscribe");
rkcg->rkcg_flags &= ~(RD_KAFKA_CGRP_F_SUBSCRIPTION |
RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* Set new atomic topic subscription.
*/
static rd_kafka_resp_err_t
rd_kafka_cgrp_subscribe(rd_kafka_cgrp_t *rkcg,
rd_kafka_topic_partition_list_t *rktparlist) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER, "SUBSCRIBE",
"Group \"%.*s\": subscribe to new %ssubscription "
"of %d topics (join-state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rktparlist ? "" : "unset ",
rktparlist ? rktparlist->cnt : 0,
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
if (rkcg->rkcg_rk->rk_conf.enabled_assignor_cnt == 0)
return RD_KAFKA_RESP_ERR__INVALID_ARG;
/* If the consumer has raised a fatal error treat all subscribes as
unsubscribe */
if (rd_kafka_fatal_error_code(rkcg->rkcg_rk)) {
if (rkcg->rkcg_subscription)
rd_kafka_cgrp_unsubscribe(rkcg,
rd_true /*leave group*/);
return RD_KAFKA_RESP_ERR__FATAL;
}
/* Clear any existing postponed subscribe. */
if (rkcg->rkcg_next_subscription)
rd_kafka_topic_partition_list_destroy_free(
rkcg->rkcg_next_subscription);
rkcg->rkcg_next_subscription = NULL;
rkcg->rkcg_next_unsubscribe = rd_false;
if (RD_KAFKA_CGRP_REBALANCING(rkcg)) {
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER, "SUBSCRIBE",
"Group \"%.*s\": postponing "
"subscribe until previous rebalance "
"completes (join-state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
if (!rktparlist)
rkcg->rkcg_next_unsubscribe = rd_true;
else
rkcg->rkcg_next_subscription = rktparlist;
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
if (rd_kafka_cgrp_rebalance_protocol(rkcg) ==
RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE &&
rktparlist && rkcg->rkcg_subscription)
return rd_kafka_cgrp_modify_subscription(rkcg, rktparlist);
/* Remove existing subscription first */
if (rkcg->rkcg_subscription)
rd_kafka_cgrp_unsubscribe(
rkcg,
rktparlist
? rd_false /* don't leave group if new subscription */
: rd_true /* leave group if no new subscription */);
if (!rktparlist)
return RD_KAFKA_RESP_ERR_NO_ERROR;
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_SUBSCRIPTION;
if (rd_kafka_topic_partition_list_regex_cnt(rktparlist) > 0)
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION;
rkcg->rkcg_subscription = rktparlist;
rd_kafka_cgrp_join(rkcg);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* Same as cgrp_terminate() but called from the cgrp/main thread upon receiving
* the op 'rko' from cgrp_terminate().
*
* NOTE: Takes ownership of 'rko'
*
* Locality: main thread
*/
void rd_kafka_cgrp_terminate0(rd_kafka_cgrp_t *rkcg, rd_kafka_op_t *rko) {
rd_kafka_assert(NULL, thrd_is_current(rkcg->rkcg_rk->rk_thread));
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPTERM",
"Terminating group \"%.*s\" in state %s "
"with %d partition(s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_list_cnt(&rkcg->rkcg_toppars));
if (unlikely(rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_TERM ||
(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE) ||
rkcg->rkcg_reply_rko != NULL)) {
/* Already terminating or handling a previous terminate */
if (rko) {
rd_kafka_q_t *rkq = rko->rko_replyq.q;
rko->rko_replyq.q = NULL;
rd_kafka_consumer_err(
rkq, RD_KAFKA_NODEID_UA,
RD_KAFKA_RESP_ERR__IN_PROGRESS,
rko->rko_replyq.version, NULL, NULL,
RD_KAFKA_OFFSET_INVALID, "Group is %s",
rkcg->rkcg_reply_rko ? "terminating"
: "terminated");
rd_kafka_q_destroy(rkq);
rd_kafka_op_destroy(rko);
}
return;
}
/* Mark for stopping, the actual state transition
* is performed when all toppars have left. */
rkcg->rkcg_flags |= RD_KAFKA_CGRP_F_TERMINATE;
rkcg->rkcg_ts_terminate = rd_clock();
rkcg->rkcg_reply_rko = rko;
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION)
rd_kafka_cgrp_unsubscribe(
rkcg,
/* Leave group if this is a controlled shutdown */
!rd_kafka_destroy_flags_no_consumer_close(rkcg->rkcg_rk));
/* Reset the wait-for-LeaveGroup flag if there is an outstanding
* LeaveGroupRequest being waited on (from a prior unsubscribe), but
* the destroy flags have NO_CONSUMER_CLOSE set, which calls
* for immediate termination. */
if (rd_kafka_destroy_flags_no_consumer_close(rkcg->rkcg_rk))
rkcg->rkcg_flags &= ~RD_KAFKA_CGRP_F_WAIT_LEAVE;
/* If there's an oustanding rebalance which has not yet been
* served by the application it will be served from consumer_close().
* If the instance is being terminated with NO_CONSUMER_CLOSE we
* trigger unassign directly to avoid stalling on rebalance callback
* queues that are no longer served by the application. */
if (!RD_KAFKA_CGRP_WAIT_ASSIGN_CALL(rkcg) ||
rd_kafka_destroy_flags_no_consumer_close(rkcg->rkcg_rk))
rd_kafka_cgrp_unassign(rkcg);
/* Serve assignment so it can start to decommission */
rd_kafka_assignment_serve(rkcg->rkcg_rk);
/* Try to terminate right away if all preconditions are met. */
rd_kafka_cgrp_try_terminate(rkcg);
}
/**
* Terminate and decommission a cgrp asynchronously.
*
* Locality: any thread
*/
void rd_kafka_cgrp_terminate(rd_kafka_cgrp_t *rkcg, rd_kafka_replyq_t replyq) {
rd_kafka_assert(NULL, !thrd_is_current(rkcg->rkcg_rk->rk_thread));
rd_kafka_cgrp_op(rkcg, NULL, replyq, RD_KAFKA_OP_TERMINATE, 0);
}
struct _op_timeout_offset_commit {
rd_ts_t now;
rd_kafka_t *rk;
rd_list_t expired;
};
/**
* q_filter callback for expiring OFFSET_COMMIT timeouts.
*/
static int rd_kafka_op_offset_commit_timeout_check(rd_kafka_q_t *rkq,
rd_kafka_op_t *rko,
void *opaque) {
struct _op_timeout_offset_commit *state =
(struct _op_timeout_offset_commit *)opaque;
if (likely(rko->rko_type != RD_KAFKA_OP_OFFSET_COMMIT ||
rko->rko_u.offset_commit.ts_timeout == 0 ||
rko->rko_u.offset_commit.ts_timeout > state->now)) {
return 0;
}
rd_kafka_q_deq0(rkq, rko);
/* Add to temporary list to avoid recursive
* locking of rkcg_wait_coord_q. */
rd_list_add(&state->expired, rko);
return 1;
}
/**
* Scan for various timeouts.
*/
static void rd_kafka_cgrp_timeout_scan(rd_kafka_cgrp_t *rkcg, rd_ts_t now) {
struct _op_timeout_offset_commit ofc_state;
int i, cnt = 0;
rd_kafka_op_t *rko;
ofc_state.now = now;
ofc_state.rk = rkcg->rkcg_rk;
rd_list_init(&ofc_state.expired, 0, NULL);
cnt += rd_kafka_q_apply(rkcg->rkcg_wait_coord_q,
rd_kafka_op_offset_commit_timeout_check,
&ofc_state);
RD_LIST_FOREACH(rko, &ofc_state.expired, i)
rd_kafka_cgrp_op_handle_OffsetCommit(rkcg->rkcg_rk, NULL,
RD_KAFKA_RESP_ERR__WAIT_COORD,
NULL, NULL, rko);
rd_list_destroy(&ofc_state.expired);
if (cnt > 0)
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "CGRPTIMEOUT",
"Group \"%.*s\": timed out %d op(s), %d remain",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id), cnt,
rd_kafka_q_len(rkcg->rkcg_wait_coord_q));
}
/**
* @brief Handle an assign op.
* @locality rdkafka main thread
* @locks none
*/
static void rd_kafka_cgrp_handle_assign_op(rd_kafka_cgrp_t *rkcg,
rd_kafka_op_t *rko) {
rd_kafka_error_t *error = NULL;
if (rd_kafka_fatal_error_code(rkcg->rkcg_rk) ||
rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE) {
/* Treat all assignments as unassign when a fatal error is
* raised or the cgrp is terminating. */
rd_kafka_dbg(rkcg->rkcg_rk, CGRP | RD_KAFKA_DBG_CONSUMER,
"ASSIGN",
"Group \"%s\": Consumer %s: "
"treating assign as unassign",
rkcg->rkcg_group_id->str,
rd_kafka_fatal_error_code(rkcg->rkcg_rk)
? "has raised a fatal error"
: "is terminating");
if (rko->rko_u.assign.partitions) {
rd_kafka_topic_partition_list_destroy(
rko->rko_u.assign.partitions);
rko->rko_u.assign.partitions = NULL;
}
rko->rko_u.assign.method = RD_KAFKA_ASSIGN_METHOD_ASSIGN;
} else if (rd_kafka_cgrp_rebalance_protocol(rkcg) ==
RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE &&
!(rko->rko_u.assign.method ==
RD_KAFKA_ASSIGN_METHOD_INCR_ASSIGN ||
rko->rko_u.assign.method ==
RD_KAFKA_ASSIGN_METHOD_INCR_UNASSIGN))
error = rd_kafka_error_new(RD_KAFKA_RESP_ERR__STATE,
"Changes to the current assignment "
"must be made using "
"incremental_assign() or "
"incremental_unassign() "
"when rebalance protocol type is "
"COOPERATIVE");
else if (rd_kafka_cgrp_rebalance_protocol(rkcg) ==
RD_KAFKA_REBALANCE_PROTOCOL_EAGER &&
!(rko->rko_u.assign.method == RD_KAFKA_ASSIGN_METHOD_ASSIGN))
error = rd_kafka_error_new(RD_KAFKA_RESP_ERR__STATE,
"Changes to the current assignment "
"must be made using "
"assign() when rebalance "
"protocol type is EAGER");
if (!error) {
switch (rko->rko_u.assign.method) {
case RD_KAFKA_ASSIGN_METHOD_ASSIGN:
/* New atomic assignment (partitions != NULL),
* or unassignment (partitions == NULL) */
if (rko->rko_u.assign.partitions)
error = rd_kafka_cgrp_assign(
rkcg, rko->rko_u.assign.partitions);
else
error = rd_kafka_cgrp_unassign(rkcg);
break;
case RD_KAFKA_ASSIGN_METHOD_INCR_ASSIGN:
error = rd_kafka_cgrp_incremental_assign(
rkcg, rko->rko_u.assign.partitions);
break;
case RD_KAFKA_ASSIGN_METHOD_INCR_UNASSIGN:
error = rd_kafka_cgrp_incremental_unassign(
rkcg, rko->rko_u.assign.partitions);
break;
default:
RD_NOTREACHED();
break;
}
/* If call succeeded serve the assignment */
if (!error)
rd_kafka_assignment_serve(rkcg->rkcg_rk);
}
if (error) {
/* Log error since caller might not check
* *assign() return value. */
rd_kafka_log(rkcg->rkcg_rk, LOG_WARNING, "ASSIGN",
"Group \"%s\": application *assign() call "
"failed: %s",
rkcg->rkcg_group_id->str,
rd_kafka_error_string(error));
}
rd_kafka_op_error_reply(rko, error);
}
/**
* @brief Handle cgrp queue op.
* @locality rdkafka main thread
* @locks none
*/
static rd_kafka_op_res_t rd_kafka_cgrp_op_serve(rd_kafka_t *rk,
rd_kafka_q_t *rkq,
rd_kafka_op_t *rko,
rd_kafka_q_cb_type_t cb_type,
void *opaque) {
rd_kafka_cgrp_t *rkcg = opaque;
rd_kafka_toppar_t *rktp;
rd_kafka_resp_err_t err;
const int silent_op = rko->rko_type == RD_KAFKA_OP_RECV_BUF;
rktp = rko->rko_rktp;
if (rktp && !silent_op)
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP, "CGRPOP",
"Group \"%.*s\" received op %s in state %s "
"(join-state %s) for %.*s [%" PRId32 "]",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_op2str(rko->rko_type),
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
rktp->rktp_partition);
else if (!silent_op)
rd_kafka_dbg(
rkcg->rkcg_rk, CGRP, "CGRPOP",
"Group \"%.*s\" received op %s in state %s "
"(join-state %s)",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_op2str(rko->rko_type),
rd_kafka_cgrp_state_names[rkcg->rkcg_state],
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
switch ((int)rko->rko_type) {
case RD_KAFKA_OP_NAME:
/* Return the currently assigned member id. */
if (rkcg->rkcg_member_id)
rko->rko_u.name.str =
RD_KAFKAP_STR_DUP(rkcg->rkcg_member_id);
rd_kafka_op_reply(rko, 0);
rko = NULL;
break;
case RD_KAFKA_OP_CG_METADATA:
/* Return the current consumer group metadata. */
rko->rko_u.cg_metadata =
rkcg->rkcg_member_id
? rd_kafka_consumer_group_metadata_new_with_genid(
rkcg->rkcg_rk->rk_conf.group_id_str,
rkcg->rkcg_generation_id,
rkcg->rkcg_member_id->str,
rkcg->rkcg_rk->rk_conf.group_instance_id)
: NULL;
rd_kafka_op_reply(rko, RD_KAFKA_RESP_ERR_NO_ERROR);
rko = NULL;
break;
case RD_KAFKA_OP_OFFSET_FETCH:
if (rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP ||
(rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE)) {
rd_kafka_op_handle_OffsetFetch(
rkcg->rkcg_rk, NULL, RD_KAFKA_RESP_ERR__WAIT_COORD,
NULL, NULL, rko);
rko = NULL; /* rko freed by handler */
break;
}
rd_kafka_OffsetFetchRequest(
rkcg->rkcg_coord, rk->rk_group_id->str,
rko->rko_u.offset_fetch.partitions,
rko->rko_u.offset_fetch.require_stable_offsets,
0, /* Timeout */
RD_KAFKA_REPLYQ(rkcg->rkcg_ops, 0),
rd_kafka_op_handle_OffsetFetch, rko);
rko = NULL; /* rko now owned by request */
break;
case RD_KAFKA_OP_PARTITION_JOIN:
rd_kafka_cgrp_partition_add(rkcg, rktp);
/* If terminating tell the partition to leave */
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_TERMINATE)
rd_kafka_toppar_op_fetch_stop(rktp, RD_KAFKA_NO_REPLYQ);
break;
case RD_KAFKA_OP_PARTITION_LEAVE:
rd_kafka_cgrp_partition_del(rkcg, rktp);
break;
case RD_KAFKA_OP_OFFSET_COMMIT:
/* Trigger offsets commit. */
rd_kafka_cgrp_offsets_commit(rkcg, rko,
/* only set offsets
* if no partitions were
* specified. */
rko->rko_u.offset_commit.partitions
? 0
: 1 /* set_offsets*/,
rko->rko_u.offset_commit.reason);
rko = NULL; /* rko now owned by request */
break;
case RD_KAFKA_OP_COORD_QUERY:
rd_kafka_cgrp_coord_query(
rkcg,
rko->rko_err ? rd_kafka_err2str(rko->rko_err) : "from op");
break;
case RD_KAFKA_OP_SUBSCRIBE:
rd_kafka_app_polled(rk);
/* New atomic subscription (may be NULL) */
err =
rd_kafka_cgrp_subscribe(rkcg, rko->rko_u.subscribe.topics);
if (!err) /* now owned by rkcg */
rko->rko_u.subscribe.topics = NULL;
rd_kafka_op_reply(rko, err);
rko = NULL;
break;
case RD_KAFKA_OP_ASSIGN:
rd_kafka_cgrp_handle_assign_op(rkcg, rko);
rko = NULL;
break;
case RD_KAFKA_OP_GET_SUBSCRIPTION:
if (rkcg->rkcg_next_subscription)
rko->rko_u.subscribe.topics =
rd_kafka_topic_partition_list_copy(
rkcg->rkcg_next_subscription);
else if (rkcg->rkcg_next_unsubscribe)
rko->rko_u.subscribe.topics = NULL;
else if (rkcg->rkcg_subscription)
rko->rko_u.subscribe.topics =
rd_kafka_topic_partition_list_copy(
rkcg->rkcg_subscription);
rd_kafka_op_reply(rko, 0);
rko = NULL;
break;
case RD_KAFKA_OP_GET_ASSIGNMENT:
/* This is the consumer assignment, not the group assignment. */
rko->rko_u.assign.partitions =
rd_kafka_topic_partition_list_copy(
rkcg->rkcg_rk->rk_consumer.assignment.all);
rd_kafka_op_reply(rko, 0);
rko = NULL;
break;
case RD_KAFKA_OP_GET_REBALANCE_PROTOCOL:
rko->rko_u.rebalance_protocol.str =
rd_kafka_rebalance_protocol2str(
rd_kafka_cgrp_rebalance_protocol(rkcg));
rd_kafka_op_reply(rko, RD_KAFKA_RESP_ERR_NO_ERROR);
rko = NULL;
break;
case RD_KAFKA_OP_TERMINATE:
rd_kafka_cgrp_terminate0(rkcg, rko);
rko = NULL; /* terminate0() takes ownership */
break;
default:
rd_kafka_assert(rkcg->rkcg_rk, !*"unknown type");
break;
}
if (rko)
rd_kafka_op_destroy(rko);
return RD_KAFKA_OP_RES_HANDLED;
}
/**
* @returns true if the session timeout has expired (due to no successful
* Heartbeats in session.timeout.ms) and triggers a rebalance.
*/
static rd_bool_t rd_kafka_cgrp_session_timeout_check(rd_kafka_cgrp_t *rkcg,
rd_ts_t now) {
rd_ts_t delta;
char buf[256];
if (unlikely(!rkcg->rkcg_ts_session_timeout))
return rd_true; /* Session has expired */
delta = now - rkcg->rkcg_ts_session_timeout;
if (likely(delta < 0))
return rd_false;
delta += rkcg->rkcg_rk->rk_conf.group_session_timeout_ms * 1000;
rd_snprintf(buf, sizeof(buf),
"Consumer group session timed out (in join-state %s) after "
"%" PRId64
" ms without a successful response from the "
"group coordinator (broker %" PRId32 ", last error was %s)",
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state],
delta / 1000, rkcg->rkcg_coord_id,
rd_kafka_err2str(rkcg->rkcg_last_heartbeat_err));
rkcg->rkcg_last_heartbeat_err = RD_KAFKA_RESP_ERR_NO_ERROR;
rd_kafka_log(rkcg->rkcg_rk, LOG_WARNING, "SESSTMOUT",
"%s: revoking assignment and rejoining group", buf);
/* Prevent further rebalances */
rkcg->rkcg_ts_session_timeout = 0;
/* Timing out invalidates the member id, reset it
* now to avoid an ERR_UNKNOWN_MEMBER_ID on the next join. */
rd_kafka_cgrp_set_member_id(rkcg, "");
/* Revoke and rebalance */
rd_kafka_cgrp_revoke_all_rejoin_maybe(rkcg, rd_true /*lost*/,
rd_true /*initiating*/, buf);
return rd_true;
}
/**
* @brief Apply the next waiting subscribe/unsubscribe, if any.
*/
static void rd_kafka_cgrp_apply_next_subscribe(rd_kafka_cgrp_t *rkcg) {
rd_assert(rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_INIT);
if (rkcg->rkcg_next_subscription) {
rd_kafka_topic_partition_list_t *next_subscription =
rkcg->rkcg_next_subscription;
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "SUBSCRIBE",
"Group \"%s\": invoking waiting postponed "
"subscribe",
rkcg->rkcg_group_id->str);
rkcg->rkcg_next_subscription = NULL;
rd_kafka_cgrp_subscribe(rkcg, next_subscription);
} else if (rkcg->rkcg_next_unsubscribe) {
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "SUBSCRIBE",
"Group \"%s\": invoking waiting postponed "
"unsubscribe",
rkcg->rkcg_group_id->str);
rkcg->rkcg_next_unsubscribe = rd_false;
rd_kafka_cgrp_unsubscribe(rkcg, rd_true /*Leave*/);
}
}
/**
* Client group's join state handling
*/
static void rd_kafka_cgrp_join_state_serve(rd_kafka_cgrp_t *rkcg) {
rd_ts_t now = rd_clock();
if (unlikely(rd_kafka_fatal_error_code(rkcg->rkcg_rk)))
return;
switch (rkcg->rkcg_join_state) {
case RD_KAFKA_CGRP_JOIN_STATE_INIT:
if (unlikely(rd_kafka_cgrp_awaiting_response(rkcg)))
break;
/* If there is a next subscription, apply it. */
rd_kafka_cgrp_apply_next_subscribe(rkcg);
/* If we have a subscription start the join process. */
if (!rkcg->rkcg_subscription)
break;
if (rd_interval_immediate(&rkcg->rkcg_join_intvl, 1000 * 1000,
now) > 0)
rd_kafka_cgrp_join(rkcg);
break;
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_JOIN:
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA:
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_SYNC:
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_TO_COMPLETE:
/* FIXME: I think we might have to send heartbeats in
* in WAIT_INCR_UNASSIGN, yes-no? */
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_INCR_UNASSIGN_TO_COMPLETE:
break;
case RD_KAFKA_CGRP_JOIN_STATE_STEADY:
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_ASSIGN_CALL:
case RD_KAFKA_CGRP_JOIN_STATE_WAIT_UNASSIGN_CALL:
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION &&
rd_interval(
&rkcg->rkcg_heartbeat_intvl,
rkcg->rkcg_rk->rk_conf.group_heartbeat_intvl_ms * 1000,
now) > 0)
rd_kafka_cgrp_heartbeat(rkcg);
break;
}
}
/**
* Client group handling.
* Called from main thread to serve the operational aspects of a cgrp.
*/
void rd_kafka_cgrp_serve(rd_kafka_cgrp_t *rkcg) {
rd_kafka_broker_t *rkb = rkcg->rkcg_coord;
int rkb_state = RD_KAFKA_BROKER_STATE_INIT;
rd_ts_t now;
if (rkb) {
rd_kafka_broker_lock(rkb);
rkb_state = rkb->rkb_state;
rd_kafka_broker_unlock(rkb);
/* Go back to querying state if we lost the current coordinator
* connection. */
if (rkb_state < RD_KAFKA_BROKER_STATE_UP &&
rkcg->rkcg_state == RD_KAFKA_CGRP_STATE_UP)
rd_kafka_cgrp_set_state(
rkcg, RD_KAFKA_CGRP_STATE_QUERY_COORD);
}
now = rd_clock();
/* Check for cgrp termination */
if (unlikely(rd_kafka_cgrp_try_terminate(rkcg))) {
rd_kafka_cgrp_terminated(rkcg);
return; /* cgrp terminated */
}
/* Bail out if we're terminating. */
if (unlikely(rd_kafka_terminating(rkcg->rkcg_rk)))
return;
/* Check session timeout regardless of current coordinator
* connection state (rkcg_state) */
if (rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY)
rd_kafka_cgrp_session_timeout_check(rkcg, now);
retry:
switch (rkcg->rkcg_state) {
case RD_KAFKA_CGRP_STATE_TERM:
break;
case RD_KAFKA_CGRP_STATE_INIT:
rd_kafka_cgrp_set_state(rkcg, RD_KAFKA_CGRP_STATE_QUERY_COORD);
/* FALLTHRU */
case RD_KAFKA_CGRP_STATE_QUERY_COORD:
/* Query for coordinator. */
if (rd_interval_immediate(&rkcg->rkcg_coord_query_intvl,
500 * 1000, now) > 0)
rd_kafka_cgrp_coord_query(rkcg,
"intervaled in "
"state query-coord");
break;
case RD_KAFKA_CGRP_STATE_WAIT_COORD:
/* Waiting for FindCoordinator response */
break;
case RD_KAFKA_CGRP_STATE_WAIT_BROKER:
/* See if the group should be reassigned to another broker. */
if (rd_kafka_cgrp_coord_update(rkcg, rkcg->rkcg_coord_id))
goto retry; /* Coordinator changed, retry state-machine
* to speed up next transition. */
/* Coordinator query */
if (rd_interval(&rkcg->rkcg_coord_query_intvl, 1000 * 1000,
now) > 0)
rd_kafka_cgrp_coord_query(rkcg,
"intervaled in "
"state wait-broker");
break;
case RD_KAFKA_CGRP_STATE_WAIT_BROKER_TRANSPORT:
/* Waiting for broker transport to come up.
* Also make sure broker supports groups. */
if (rkb_state < RD_KAFKA_BROKER_STATE_UP || !rkb ||
!rd_kafka_broker_supports(
rkb, RD_KAFKA_FEATURE_BROKER_GROUP_COORD)) {
/* Coordinator query */
if (rd_interval(&rkcg->rkcg_coord_query_intvl,
1000 * 1000, now) > 0)
rd_kafka_cgrp_coord_query(
rkcg,
"intervaled in state "
"wait-broker-transport");
} else {
rd_kafka_cgrp_set_state(rkcg, RD_KAFKA_CGRP_STATE_UP);
/* Serve join state to trigger (re)join */
rd_kafka_cgrp_join_state_serve(rkcg);
/* Serve any pending partitions in the assignment */
rd_kafka_assignment_serve(rkcg->rkcg_rk);
}
break;
case RD_KAFKA_CGRP_STATE_UP:
/* Move any ops awaiting the coordinator to the ops queue
* for reprocessing. */
rd_kafka_q_concat(rkcg->rkcg_ops, rkcg->rkcg_wait_coord_q);
/* Relaxed coordinator queries. */
if (rd_interval(&rkcg->rkcg_coord_query_intvl,
rkcg->rkcg_rk->rk_conf.coord_query_intvl_ms *
1000,
now) > 0)
rd_kafka_cgrp_coord_query(rkcg,
"intervaled in state up");
rd_kafka_cgrp_join_state_serve(rkcg);
break;
}
if (unlikely(rkcg->rkcg_state != RD_KAFKA_CGRP_STATE_UP &&
rd_interval(&rkcg->rkcg_timeout_scan_intvl, 1000 * 1000,
now) > 0))
rd_kafka_cgrp_timeout_scan(rkcg, now);
}
/**
* Send an op to a cgrp.
*
* Locality: any thread
*/
void rd_kafka_cgrp_op(rd_kafka_cgrp_t *rkcg,
rd_kafka_toppar_t *rktp,
rd_kafka_replyq_t replyq,
rd_kafka_op_type_t type,
rd_kafka_resp_err_t err) {
rd_kafka_op_t *rko;
rko = rd_kafka_op_new(type);
rko->rko_err = err;
rko->rko_replyq = replyq;
if (rktp)
rko->rko_rktp = rd_kafka_toppar_keep(rktp);
rd_kafka_q_enq(rkcg->rkcg_ops, rko);
}
void rd_kafka_cgrp_set_member_id(rd_kafka_cgrp_t *rkcg, const char *member_id) {
if (rkcg->rkcg_member_id && member_id &&
!rd_kafkap_str_cmp_str(rkcg->rkcg_member_id, member_id))
return; /* No change */
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "MEMBERID",
"Group \"%.*s\": updating member id \"%s\" -> \"%s\"",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rkcg->rkcg_member_id ? rkcg->rkcg_member_id->str
: "(not-set)",
member_id ? member_id : "(not-set)");
if (rkcg->rkcg_member_id) {
rd_kafkap_str_destroy(rkcg->rkcg_member_id);
rkcg->rkcg_member_id = NULL;
}
if (member_id)
rkcg->rkcg_member_id = rd_kafkap_str_new(member_id, -1);
}
/**
* @brief Determine owned partitions that no longer exist (partitions in
* deleted or re-created topics).
*/
static rd_kafka_topic_partition_list_t *
rd_kafka_cgrp_owned_but_not_exist_partitions(rd_kafka_cgrp_t *rkcg) {
rd_kafka_topic_partition_list_t *result = NULL;
const rd_kafka_topic_partition_t *curr;
if (!rkcg->rkcg_group_assignment)
return NULL;
RD_KAFKA_TPLIST_FOREACH(curr, rkcg->rkcg_group_assignment) {
if (rd_list_find(rkcg->rkcg_subscribed_topics, curr->topic,
rd_kafka_topic_info_topic_cmp))
continue;
if (!result)
result = rd_kafka_topic_partition_list_new(
rkcg->rkcg_group_assignment->cnt);
rd_kafka_topic_partition_list_add_copy(result, curr);
}
return result;
}
/**
* @brief Check if the latest metadata affects the current subscription:
* - matched topic added
* - matched topic removed
* - matched topic's partition count change
*
* @locks none
* @locality rdkafka main thread
*/
void rd_kafka_cgrp_metadata_update_check(rd_kafka_cgrp_t *rkcg,
rd_bool_t do_join) {
rd_list_t *tinfos;
rd_kafka_topic_partition_list_t *errored;
rd_bool_t changed;
rd_kafka_assert(NULL, thrd_is_current(rkcg->rkcg_rk->rk_thread));
if (!rkcg->rkcg_subscription || rkcg->rkcg_subscription->cnt == 0)
return;
/*
* Unmatched topics will be added to the errored list.
*/
errored = rd_kafka_topic_partition_list_new(0);
/*
* Create a list of the topics in metadata that matches our subscription
*/
tinfos = rd_list_new(rkcg->rkcg_subscription->cnt,
(void *)rd_kafka_topic_info_destroy);
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_WILDCARD_SUBSCRIPTION)
rd_kafka_metadata_topic_match(rkcg->rkcg_rk, tinfos,
rkcg->rkcg_subscription, errored);
else
rd_kafka_metadata_topic_filter(
rkcg->rkcg_rk, tinfos, rkcg->rkcg_subscription, errored);
/*
* Propagate consumer errors for any non-existent or errored topics.
* The function takes ownership of errored.
*/
rd_kafka_propagate_consumer_topic_errors(
rkcg, errored, "Subscribed topic not available");
/*
* Update effective list of topics (takes ownership of \c tinfos)
*/
changed = rd_kafka_cgrp_update_subscribed_topics(rkcg, tinfos);
if (!do_join ||
(!changed &&
/* If we get the same effective list of topics as last time around,
* but the join is waiting for this metadata query to complete,
* then we should not return here but follow through with the
* (re)join below. */
rkcg->rkcg_join_state != RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA))
return;
/* List of subscribed topics changed, trigger rejoin. */
rd_kafka_dbg(rkcg->rkcg_rk,
CGRP | RD_KAFKA_DBG_METADATA | RD_KAFKA_DBG_CONSUMER,
"REJOIN",
"Group \"%.*s\": "
"subscription updated from metadata change: "
"rejoining group in state %s",
RD_KAFKAP_STR_PR(rkcg->rkcg_group_id),
rd_kafka_cgrp_join_state_names[rkcg->rkcg_join_state]);
if (rd_kafka_cgrp_rebalance_protocol(rkcg) ==
RD_KAFKA_REBALANCE_PROTOCOL_COOPERATIVE) {
/* Partitions from deleted topics */
rd_kafka_topic_partition_list_t *owned_but_not_exist =
rd_kafka_cgrp_owned_but_not_exist_partitions(rkcg);
if (owned_but_not_exist) {
rd_kafka_cgrp_assignment_set_lost(
rkcg, "%d subscribed topic(s) no longer exist",
owned_but_not_exist->cnt);
rd_kafka_rebalance_op_incr(
rkcg, RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS,
owned_but_not_exist,
rkcg->rkcg_group_leader.members != NULL
/* Rejoin group following revoke's
* unassign if we are leader */
,
"topics not available");
rd_kafka_topic_partition_list_destroy(
owned_but_not_exist);
} else {
/* Nothing to revoke, rejoin group if we are the
* leader.
* The KIP says to rejoin the group on metadata
* change only if we're the leader. But what if a
* non-leader is subscribed to a regex that the others
* aren't?
* Going against the KIP and rejoining here. */
rd_kafka_cgrp_rejoin(
rkcg,
"Metadata for subscribed topic(s) has "
"changed");
}
} else {
/* EAGER */
rd_kafka_cgrp_revoke_rejoin(rkcg,
"Metadata for subscribed topic(s) "
"has changed");
}
/* We shouldn't get stuck in this state. */
rd_dassert(rkcg->rkcg_join_state !=
RD_KAFKA_CGRP_JOIN_STATE_WAIT_METADATA);
}
rd_kafka_consumer_group_metadata_t *
rd_kafka_consumer_group_metadata_new(const char *group_id) {
rd_kafka_consumer_group_metadata_t *cgmetadata;
cgmetadata = rd_kafka_consumer_group_metadata_new_with_genid(
group_id, -1, "", NULL);
return cgmetadata;
}
rd_kafka_consumer_group_metadata_t *
rd_kafka_consumer_group_metadata_new_with_genid(const char *group_id,
int32_t generation_id,
const char *member_id,
const char *group_instance_id) {
rd_kafka_consumer_group_metadata_t *cgmetadata;
cgmetadata = rd_calloc(1, sizeof(*cgmetadata));
cgmetadata->group_id = rd_strdup(group_id);
cgmetadata->generation_id = generation_id;
cgmetadata->member_id = rd_strdup(member_id);
if (group_instance_id)
cgmetadata->group_instance_id = rd_strdup(group_instance_id);
return cgmetadata;
}
rd_kafka_consumer_group_metadata_t *
rd_kafka_consumer_group_metadata(rd_kafka_t *rk) {
rd_kafka_consumer_group_metadata_t *cgmetadata;
rd_kafka_op_t *rko;
rd_kafka_cgrp_t *rkcg;
if (!(rkcg = rd_kafka_cgrp_get(rk)))
return NULL;
rko = rd_kafka_op_req2(rkcg->rkcg_ops, RD_KAFKA_OP_CG_METADATA);
if (!rko)
return NULL;
cgmetadata = rko->rko_u.cg_metadata;
rko->rko_u.cg_metadata = NULL;
rd_kafka_op_destroy(rko);
return cgmetadata;
}
void rd_kafka_consumer_group_metadata_destroy(
rd_kafka_consumer_group_metadata_t *cgmetadata) {
rd_free(cgmetadata->group_id);
rd_free(cgmetadata->member_id);
if (cgmetadata->group_instance_id)
rd_free(cgmetadata->group_instance_id);
rd_free(cgmetadata);
}
rd_kafka_consumer_group_metadata_t *rd_kafka_consumer_group_metadata_dup(
const rd_kafka_consumer_group_metadata_t *cgmetadata) {
rd_kafka_consumer_group_metadata_t *ret;
ret = rd_calloc(1, sizeof(*cgmetadata));
ret->group_id = rd_strdup(cgmetadata->group_id);
ret->generation_id = cgmetadata->generation_id;
ret->member_id = rd_strdup(cgmetadata->member_id);
if (cgmetadata->group_instance_id)
ret->group_instance_id =
rd_strdup(cgmetadata->group_instance_id);
return ret;
}
/*
* Consumer group metadata serialization format v2:
* "CGMDv2:"<generation_id><group_id>"\0"<member_id>"\0" \
* <group_instance_id_is_null>[<group_instance_id>"\0"]
* Where <group_id> is the group_id string.
*/
static const char rd_kafka_consumer_group_metadata_magic[7] = "CGMDv2:";
rd_kafka_error_t *rd_kafka_consumer_group_metadata_write(
const rd_kafka_consumer_group_metadata_t *cgmd,
void **bufferp,
size_t *sizep) {
char *buf;
size_t size;
size_t of = 0;
size_t magic_len = sizeof(rd_kafka_consumer_group_metadata_magic);
size_t groupid_len = strlen(cgmd->group_id) + 1;
size_t generationid_len = sizeof(cgmd->generation_id);
size_t member_id_len = strlen(cgmd->member_id) + 1;
int8_t group_instance_id_is_null = cgmd->group_instance_id ? 0 : 1;
size_t group_instance_id_is_null_len =
sizeof(group_instance_id_is_null);
size_t group_instance_id_len =
cgmd->group_instance_id ? strlen(cgmd->group_instance_id) + 1 : 0;
size = magic_len + groupid_len + generationid_len + member_id_len +
group_instance_id_is_null_len + group_instance_id_len;
buf = rd_malloc(size);
memcpy(buf, rd_kafka_consumer_group_metadata_magic, magic_len);
of += magic_len;
memcpy(buf + of, &cgmd->generation_id, generationid_len);
of += generationid_len;
memcpy(buf + of, cgmd->group_id, groupid_len);
of += groupid_len;
memcpy(buf + of, cgmd->member_id, member_id_len);
of += member_id_len;
memcpy(buf + of, &group_instance_id_is_null,
group_instance_id_is_null_len);
of += group_instance_id_is_null_len;
if (!group_instance_id_is_null)
memcpy(buf + of, cgmd->group_instance_id,
group_instance_id_len);
of += group_instance_id_len;
rd_assert(of == size);
*bufferp = buf;
*sizep = size;
return NULL;
}
/*
* Check that a string is printable, returning NULL if not or
* a pointer immediately after the end of the string NUL
* terminator if so.
**/
static const char *str_is_printable(const char *s, const char *end) {
const char *c;
for (c = s; *c && c != end; c++)
if (!isprint((int)*c))
return NULL;
return c + 1;
}
rd_kafka_error_t *rd_kafka_consumer_group_metadata_read(
rd_kafka_consumer_group_metadata_t **cgmdp,
const void *buffer,
size_t size) {
const char *buf = (const char *)buffer;
const char *end = buf + size;
const char *next;
size_t magic_len = sizeof(rd_kafka_consumer_group_metadata_magic);
int32_t generation_id;
size_t generationid_len = sizeof(generation_id);
const char *group_id;
const char *member_id;
int8_t group_instance_id_is_null;
const char *group_instance_id = NULL;
if (size < magic_len + generationid_len + 1 + 1 + 1)
return rd_kafka_error_new(RD_KAFKA_RESP_ERR__BAD_MSG,
"Input buffer is too short");
if (memcmp(buffer, rd_kafka_consumer_group_metadata_magic, magic_len))
return rd_kafka_error_new(RD_KAFKA_RESP_ERR__BAD_MSG,
"Input buffer is not a serialized "
"consumer group metadata object");
memcpy(&generation_id, buf + magic_len, generationid_len);
group_id = buf + magic_len + generationid_len;
next = str_is_printable(group_id, end);
if (!next)
return rd_kafka_error_new(RD_KAFKA_RESP_ERR__BAD_MSG,
"Input buffer group id is not safe");
member_id = next;
next = str_is_printable(member_id, end);
if (!next)
return rd_kafka_error_new(RD_KAFKA_RESP_ERR__BAD_MSG,
"Input buffer member id is not "
"safe");
group_instance_id_is_null = (int8_t) * (next++);
if (!group_instance_id_is_null) {
group_instance_id = next;
next = str_is_printable(group_instance_id, end);
if (!next)
return rd_kafka_error_new(RD_KAFKA_RESP_ERR__BAD_MSG,
"Input buffer group "
"instance id is not safe");
}
if (next != end)
return rd_kafka_error_new(RD_KAFKA_RESP_ERR__BAD_MSG,
"Input buffer bad length");
*cgmdp = rd_kafka_consumer_group_metadata_new_with_genid(
group_id, generation_id, member_id, group_instance_id);
return NULL;
}
static int
unittest_consumer_group_metadata_iteration(const char *group_id,
int32_t generation_id,
const char *member_id,
const char *group_instance_id) {
rd_kafka_consumer_group_metadata_t *cgmd;
void *buffer, *buffer2;
size_t size, size2;
rd_kafka_error_t *error;
cgmd = rd_kafka_consumer_group_metadata_new_with_genid(
group_id, generation_id, member_id, group_instance_id);
RD_UT_ASSERT(cgmd != NULL, "failed to create metadata");
error = rd_kafka_consumer_group_metadata_write(cgmd, &buffer, &size);
RD_UT_ASSERT(!error, "metadata_write failed: %s",
rd_kafka_error_string(error));
rd_kafka_consumer_group_metadata_destroy(cgmd);
cgmd = NULL;
error = rd_kafka_consumer_group_metadata_read(&cgmd, buffer, size);
RD_UT_ASSERT(!error, "metadata_read failed: %s",
rd_kafka_error_string(error));
/* Serialize again and compare buffers */
error = rd_kafka_consumer_group_metadata_write(cgmd, &buffer2, &size2);
RD_UT_ASSERT(!error, "metadata_write failed: %s",
rd_kafka_error_string(error));
RD_UT_ASSERT(size == size2 && !memcmp(buffer, buffer2, size),
"metadata_read/write size or content mismatch: "
"size %" PRIusz ", size2 %" PRIusz,
size, size2);
rd_kafka_consumer_group_metadata_destroy(cgmd);
rd_free(buffer);
rd_free(buffer2);
return 0;
}
static int unittest_consumer_group_metadata(void) {
const char *ids[] = {
"mY. random id:.",
"0",
"2222222222222222222222221111111111111111111111111111112222",
"",
"NULL",
NULL,
};
int i, j, k, gen_id;
int ret;
const char *group_id;
const char *member_id;
const char *group_instance_id;
for (i = 0; ids[i]; i++) {
for (j = 0; ids[j]; j++) {
for (k = 0; ids[k]; k++) {
for (gen_id = -1; gen_id < 1; gen_id++) {
group_id = ids[i];
member_id = ids[j];
group_instance_id = ids[k];
if (strcmp(group_instance_id, "NULL") ==
0)
group_instance_id = NULL;
ret =
unittest_consumer_group_metadata_iteration(
group_id, gen_id, member_id,
group_instance_id);
if (ret)
return ret;
}
}
}
}
RD_UT_PASS();
}
static int unittest_set_intersect(void) {
size_t par_cnt = 10;
map_toppar_member_info_t *dst;
rd_kafka_topic_partition_t *toppar;
PartitionMemberInfo_t *v;
char *id = "id";
rd_kafkap_str_t id1 = RD_KAFKAP_STR_INITIALIZER;
rd_kafkap_str_t id2 = RD_KAFKAP_STR_INITIALIZER;
rd_kafka_group_member_t *gm1;
rd_kafka_group_member_t *gm2;
id1.len = 2;
id1.str = id;
id2.len = 2;
id2.str = id;
map_toppar_member_info_t a = RD_MAP_INITIALIZER(
par_cnt, rd_kafka_topic_partition_cmp,
rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free, PartitionMemberInfo_free);
map_toppar_member_info_t b = RD_MAP_INITIALIZER(
par_cnt, rd_kafka_topic_partition_cmp,
rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free, PartitionMemberInfo_free);
gm1 = rd_calloc(1, sizeof(*gm1));
gm1->rkgm_member_id = &id1;
gm1->rkgm_group_instance_id = &id1;
gm2 = rd_calloc(1, sizeof(*gm2));
gm2->rkgm_member_id = &id2;
gm2->rkgm_group_instance_id = &id2;
RD_MAP_SET(&a, rd_kafka_topic_partition_new("t1", 4),
PartitionMemberInfo_new(gm1, rd_false));
RD_MAP_SET(&a, rd_kafka_topic_partition_new("t2", 4),
PartitionMemberInfo_new(gm1, rd_false));
RD_MAP_SET(&a, rd_kafka_topic_partition_new("t1", 7),
PartitionMemberInfo_new(gm1, rd_false));
RD_MAP_SET(&b, rd_kafka_topic_partition_new("t2", 7),
PartitionMemberInfo_new(gm1, rd_false));
RD_MAP_SET(&b, rd_kafka_topic_partition_new("t1", 4),
PartitionMemberInfo_new(gm2, rd_false));
dst = rd_kafka_member_partitions_intersect(&a, &b);
RD_UT_ASSERT(RD_MAP_CNT(&a) == 3, "expected a cnt to be 3 not %d",
(int)RD_MAP_CNT(&a));
RD_UT_ASSERT(RD_MAP_CNT(&b) == 2, "expected b cnt to be 2 not %d",
(int)RD_MAP_CNT(&b));
RD_UT_ASSERT(RD_MAP_CNT(dst) == 1, "expected dst cnt to be 1 not %d",
(int)RD_MAP_CNT(dst));
toppar = rd_kafka_topic_partition_new("t1", 4);
RD_UT_ASSERT((v = RD_MAP_GET(dst, toppar)), "unexpected element");
RD_UT_ASSERT(v->members_match, "expected members to match");
rd_kafka_topic_partition_destroy(toppar);
RD_MAP_DESTROY(&a);
RD_MAP_DESTROY(&b);
RD_MAP_DESTROY(dst);
rd_free(dst);
rd_free(gm1);
rd_free(gm2);
RD_UT_PASS();
}
static int unittest_set_subtract(void) {
size_t par_cnt = 10;
rd_kafka_topic_partition_t *toppar;
map_toppar_member_info_t *dst;
map_toppar_member_info_t a = RD_MAP_INITIALIZER(
par_cnt, rd_kafka_topic_partition_cmp,
rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free, PartitionMemberInfo_free);
map_toppar_member_info_t b = RD_MAP_INITIALIZER(
par_cnt, rd_kafka_topic_partition_cmp,
rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free, PartitionMemberInfo_free);
RD_MAP_SET(&a, rd_kafka_topic_partition_new("t1", 4),
PartitionMemberInfo_new(NULL, rd_false));
RD_MAP_SET(&a, rd_kafka_topic_partition_new("t2", 7),
PartitionMemberInfo_new(NULL, rd_false));
RD_MAP_SET(&b, rd_kafka_topic_partition_new("t2", 4),
PartitionMemberInfo_new(NULL, rd_false));
RD_MAP_SET(&b, rd_kafka_topic_partition_new("t1", 4),
PartitionMemberInfo_new(NULL, rd_false));
RD_MAP_SET(&b, rd_kafka_topic_partition_new("t1", 7),
PartitionMemberInfo_new(NULL, rd_false));
dst = rd_kafka_member_partitions_subtract(&a, &b);
RD_UT_ASSERT(RD_MAP_CNT(&a) == 2, "expected a cnt to be 2 not %d",
(int)RD_MAP_CNT(&a));
RD_UT_ASSERT(RD_MAP_CNT(&b) == 3, "expected b cnt to be 3 not %d",
(int)RD_MAP_CNT(&b));
RD_UT_ASSERT(RD_MAP_CNT(dst) == 1, "expected dst cnt to be 1 not %d",
(int)RD_MAP_CNT(dst));
toppar = rd_kafka_topic_partition_new("t2", 7);
RD_UT_ASSERT(RD_MAP_GET(dst, toppar), "unexpected element");
rd_kafka_topic_partition_destroy(toppar);
RD_MAP_DESTROY(&a);
RD_MAP_DESTROY(&b);
RD_MAP_DESTROY(dst);
rd_free(dst);
RD_UT_PASS();
}
static int unittest_map_to_list(void) {
rd_kafka_topic_partition_list_t *list;
map_toppar_member_info_t map = RD_MAP_INITIALIZER(
10, rd_kafka_topic_partition_cmp, rd_kafka_topic_partition_hash,
rd_kafka_topic_partition_destroy_free, PartitionMemberInfo_free);
RD_MAP_SET(&map, rd_kafka_topic_partition_new("t1", 101),
PartitionMemberInfo_new(NULL, rd_false));
list = rd_kafka_toppar_member_info_map_to_list(&map);
RD_UT_ASSERT(list->cnt == 1, "expecting list size of 1 not %d.",
list->cnt);
RD_UT_ASSERT(list->elems[0].partition == 101,
"expecting partition 101 not %d",
list->elems[0].partition);
RD_UT_ASSERT(!strcmp(list->elems[0].topic, "t1"),
"expecting topic 't1', not %s", list->elems[0].topic);
rd_kafka_topic_partition_list_destroy(list);
RD_MAP_DESTROY(&map);
RD_UT_PASS();
}
static int unittest_list_to_map(void) {
rd_kafka_topic_partition_t *toppar;
map_toppar_member_info_t *map;
rd_kafka_topic_partition_list_t *list =
rd_kafka_topic_partition_list_new(1);
rd_kafka_topic_partition_list_add(list, "topic1", 201);
rd_kafka_topic_partition_list_add(list, "topic2", 202);
map = rd_kafka_toppar_list_to_toppar_member_info_map(list);
RD_UT_ASSERT(RD_MAP_CNT(map) == 2, "expected map cnt to be 2 not %d",
(int)RD_MAP_CNT(map));
toppar = rd_kafka_topic_partition_new("topic1", 201);
RD_UT_ASSERT(RD_MAP_GET(map, toppar),
"expected topic1 [201] to exist in map");
rd_kafka_topic_partition_destroy(toppar);
toppar = rd_kafka_topic_partition_new("topic2", 202);
RD_UT_ASSERT(RD_MAP_GET(map, toppar),
"expected topic2 [202] to exist in map");
rd_kafka_topic_partition_destroy(toppar);
RD_MAP_DESTROY(map);
rd_free(map);
rd_kafka_topic_partition_list_destroy(list);
RD_UT_PASS();
}
/**
* @brief Consumer group unit tests
*/
int unittest_cgrp(void) {
int fails = 0;
fails += unittest_consumer_group_metadata();
fails += unittest_set_intersect();
fails += unittest_set_subtract();
fails += unittest_map_to_list();
fails += unittest_list_to_map();
return fails;
}
|