datafusion-physical-plan 53.1.0

Physical (ExecutionPlan) implementations for DataFusion query engine
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

//! This file implements the [`RepartitionExec`]  operator, which maps N input
//! partitions to M output partitions based on a partitioning scheme, optionally
//! maintaining the order of the input rows in the output.

use std::fmt::{Debug, Formatter};
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use std::{any::Any, vec};

use super::common::SharedMemoryReservation;
use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet};
use super::{
    DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream,
};
use crate::coalesce::LimitedBatchCoalescer;
use crate::execution_plan::{CardinalityEffect, EvaluationType, SchedulingType};
use crate::hash_utils::create_hashes;
use crate::metrics::{BaselineMetrics, SpillMetrics};
use crate::projection::{ProjectionExec, all_columns, make_with_child, update_expr};
use crate::sorts::streaming_merge::StreamingMergeBuilder;
use crate::spill::spill_manager::SpillManager;
use crate::spill::spill_pool::{self, SpillPoolWriter};
use crate::stream::RecordBatchStreamAdapter;
use crate::{
    DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics,
    check_if_same_properties,
};

use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions};
use arrow::compute::take_arrays;
use arrow::datatypes::{SchemaRef, UInt32Type};
use datafusion_common::config::ConfigOptions;
use datafusion_common::stats::Precision;
use datafusion_common::utils::transpose;
use datafusion_common::{
    ColumnStatistics, DataFusionError, HashMap, assert_or_internal_err,
    internal_datafusion_err, internal_err,
};
use datafusion_common::{Result, not_impl_err};
use datafusion_common_runtime::SpawnedTask;
use datafusion_execution::TaskContext;
use datafusion_execution::memory_pool::MemoryConsumer;
use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr};
use datafusion_physical_expr_common::sort_expr::LexOrdering;

use crate::filter_pushdown::{
    ChildPushdownResult, FilterDescription, FilterPushdownPhase,
    FilterPushdownPropagation,
};
use crate::joins::SeededRandomState;
use crate::sort_pushdown::SortOrderPushdownResult;
use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr;
use datafusion_physical_expr_common::utils::evaluate_expressions_to_arrays;
use futures::stream::Stream;
use futures::{FutureExt, StreamExt, TryStreamExt, ready};
use log::trace;
use parking_lot::Mutex;

mod distributor_channels;
use distributor_channels::{
    DistributionReceiver, DistributionSender, channels, partition_aware_channels,
};

/// A batch in the repartition queue - either in memory or spilled to disk.
///
/// This enum represents the two states a batch can be in during repartitioning.
/// The decision to spill is made based on memory availability when sending a batch
/// to an output partition.
///
/// # Batch Flow with Spilling
///
/// ```text
/// Input Stream ──▶ Partition Logic ──▶ try_grow()
//////                            ┌───────────────┴────────────────┐
///                            │                                │
///                            ▼                                ▼
///                   try_grow() succeeds            try_grow() fails
///                   (Memory Available)              (Memory Pressure)
///                            │                                │
///                            ▼                                ▼
///                  RepartitionBatch::Memory         spill_writer.push_batch()
///                  (batch held in memory)           (batch written to disk)
///                            │                                │
///                            │                                ▼
///                            │                      RepartitionBatch::Spilled
///                            │                      (marker - no batch data)
///                            │                                │
///                            └────────┬───────────────────────┘
/////////                              Send to channel
/////////                            Output Stream (poll)
//////                      ┌──────────────┴─────────────┐
///                      │                            │
///                      ▼                            ▼
///         RepartitionBatch::Memory      RepartitionBatch::Spilled
///         Return batch immediately       Poll spill_stream (blocks)
///                      │                            │
///                      └────────┬───────────────────┘
/////////                          Return batch
///                    (FIFO order preserved)
/// ```
///
/// See [`RepartitionExec`] for overall architecture and [`StreamState`] for
/// the state machine that handles reading these batches.
#[derive(Debug)]
enum RepartitionBatch {
    /// Batch held in memory (counts against memory reservation)
    Memory(RecordBatch),
    /// Marker indicating a batch was spilled to the partition's SpillPool.
    /// The actual batch can be retrieved by reading from the SpillPoolStream.
    /// This variant contains no data itself - it's just a signal to the reader
    /// to fetch the next batch from the spill stream.
    Spilled,
}

type MaybeBatch = Option<Result<RepartitionBatch>>;
type InputPartitionsToCurrentPartitionSender = Vec<DistributionSender<MaybeBatch>>;
type InputPartitionsToCurrentPartitionReceiver = Vec<DistributionReceiver<MaybeBatch>>;

/// Output channel with its associated memory reservation and spill writer
struct OutputChannel {
    sender: DistributionSender<MaybeBatch>,
    reservation: SharedMemoryReservation,
    spill_writer: SpillPoolWriter,
}

/// Channels and resources for a single output partition.
///
/// Each output partition has channels to receive data from all input partitions.
/// To handle memory pressure, each (input, output) pair gets its own
/// [`SpillPool`](crate::spill::spill_pool) channel via [`spill_pool::channel`].
///
/// # Structure
///
/// For an output partition receiving from N input partitions:
/// - `tx`: N senders (one per input) for sending batches to this output
/// - `rx`: N receivers (one per input) for receiving batches at this output
/// - `spill_writers`: N spill writers (one per input) for writing spilled data
/// - `spill_readers`: N spill readers (one per input) for reading spilled data
///
/// This 1:1 mapping between input partitions and spill channels ensures that
/// batches from each input are processed in FIFO order, even when some batches
/// are spilled to disk and others remain in memory.
///
/// See [`RepartitionExec`] for the overall N×M architecture.
///
/// [`spill_pool::channel`]: crate::spill::spill_pool::channel
struct PartitionChannels {
    /// Senders for each input partition to send data to this output partition
    tx: InputPartitionsToCurrentPartitionSender,
    /// Receivers for each input partition sending data to this output partition
    rx: InputPartitionsToCurrentPartitionReceiver,
    /// Memory reservation for this output partition
    reservation: SharedMemoryReservation,
    /// Spill writers for writing spilled data.
    /// SpillPoolWriter is Clone, so multiple writers can share state in non-preserve-order mode.
    spill_writers: Vec<SpillPoolWriter>,
    /// Spill readers for reading spilled data - one per input partition (FIFO semantics).
    /// Each (input, output) pair gets its own reader to maintain proper ordering.
    spill_readers: Vec<SendableRecordBatchStream>,
}

struct ConsumingInputStreamsState {
    /// Channels for sending batches from input partitions to output partitions.
    /// Key is the partition number.
    channels: HashMap<usize, PartitionChannels>,

    /// Helper that ensures that background jobs are killed once they are no longer needed.
    abort_helper: Arc<Vec<SpawnedTask<()>>>,
}

impl Debug for ConsumingInputStreamsState {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ConsumingInputStreamsState")
            .field("num_channels", &self.channels.len())
            .field("abort_helper", &self.abort_helper)
            .finish()
    }
}

/// Inner state of [`RepartitionExec`].
#[derive(Default)]
enum RepartitionExecState {
    /// Not initialized yet. This is the default state stored in the RepartitionExec node
    /// upon instantiation.
    #[default]
    NotInitialized,
    /// Input streams are initialized, but they are still not being consumed. The node
    /// transitions to this state when the arrow's RecordBatch stream is created in
    /// RepartitionExec::execute(), but before any message is polled.
    InputStreamsInitialized(Vec<(SendableRecordBatchStream, RepartitionMetrics)>),
    /// The input streams are being consumed. The node transitions to this state when
    /// the first message in the arrow's RecordBatch stream is consumed.
    ConsumingInputStreams(ConsumingInputStreamsState),
}

impl Debug for RepartitionExecState {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        match self {
            RepartitionExecState::NotInitialized => write!(f, "NotInitialized"),
            RepartitionExecState::InputStreamsInitialized(v) => {
                write!(f, "InputStreamsInitialized({:?})", v.len())
            }
            RepartitionExecState::ConsumingInputStreams(v) => {
                write!(f, "ConsumingInputStreams({v:?})")
            }
        }
    }
}

impl RepartitionExecState {
    fn ensure_input_streams_initialized(
        &mut self,
        input: &Arc<dyn ExecutionPlan>,
        metrics: &ExecutionPlanMetricsSet,
        output_partitions: usize,
        ctx: &Arc<TaskContext>,
    ) -> Result<()> {
        if !matches!(self, RepartitionExecState::NotInitialized) {
            return Ok(());
        }

        let num_input_partitions = input.output_partitioning().partition_count();
        let mut streams_and_metrics = Vec::with_capacity(num_input_partitions);

        for i in 0..num_input_partitions {
            let metrics = RepartitionMetrics::new(i, output_partitions, metrics);

            let timer = metrics.fetch_time.timer();
            let stream = input.execute(i, Arc::clone(ctx))?;
            timer.done();

            streams_and_metrics.push((stream, metrics));
        }
        *self = RepartitionExecState::InputStreamsInitialized(streams_and_metrics);
        Ok(())
    }

    #[expect(clippy::too_many_arguments)]
    fn consume_input_streams(
        &mut self,
        input: &Arc<dyn ExecutionPlan>,
        metrics: &ExecutionPlanMetricsSet,
        partitioning: &Partitioning,
        preserve_order: bool,
        name: &str,
        context: &Arc<TaskContext>,
        spill_manager: SpillManager,
    ) -> Result<&mut ConsumingInputStreamsState> {
        let streams_and_metrics = match self {
            RepartitionExecState::NotInitialized => {
                self.ensure_input_streams_initialized(
                    input,
                    metrics,
                    partitioning.partition_count(),
                    context,
                )?;
                let RepartitionExecState::InputStreamsInitialized(value) = self else {
                    // This cannot happen, as ensure_input_streams_initialized() was just called,
                    // but the compiler does not know.
                    return internal_err!(
                        "Programming error: RepartitionExecState must be in the InputStreamsInitialized state after calling RepartitionExecState::ensure_input_streams_initialized"
                    );
                };
                value
            }
            RepartitionExecState::ConsumingInputStreams(value) => return Ok(value),
            RepartitionExecState::InputStreamsInitialized(value) => value,
        };

        let num_input_partitions = streams_and_metrics.len();
        let num_output_partitions = partitioning.partition_count();

        let spill_manager = Arc::new(spill_manager);

        let (txs, rxs) = if preserve_order {
            // Create partition-aware channels with one channel per (input, output) pair
            // This provides backpressure while maintaining proper ordering
            let (txs_all, rxs_all) =
                partition_aware_channels(num_input_partitions, num_output_partitions);
            // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition
            let txs = transpose(txs_all);
            let rxs = transpose(rxs_all);
            (txs, rxs)
        } else {
            // Create one channel per *output* partition with backpressure
            let (txs, rxs) = channels(num_output_partitions);
            // Clone sender for each input partitions
            let txs = txs
                .into_iter()
                .map(|item| vec![item; num_input_partitions])
                .collect::<Vec<_>>();
            let rxs = rxs.into_iter().map(|item| vec![item]).collect::<Vec<_>>();
            (txs, rxs)
        };

        let mut channels = HashMap::with_capacity(txs.len());
        for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() {
            let reservation = Arc::new(Mutex::new(
                MemoryConsumer::new(format!("{name}[{partition}]"))
                    .with_can_spill(true)
                    .register(context.memory_pool()),
            ));

            // Create spill channels based on mode:
            // - preserve_order: one spill channel per (input, output) pair for proper FIFO ordering
            // - non-preserve-order: one shared spill channel per output partition since all inputs
            //   share the same receiver
            let max_file_size = context
                .session_config()
                .options()
                .execution
                .max_spill_file_size_bytes;
            let num_spill_channels = if preserve_order {
                num_input_partitions
            } else {
                1
            };
            let (spill_writers, spill_readers): (Vec<_>, Vec<_>) = (0
                ..num_spill_channels)
                .map(|_| spill_pool::channel(max_file_size, Arc::clone(&spill_manager)))
                .unzip();

            channels.insert(
                partition,
                PartitionChannels {
                    tx,
                    rx,
                    reservation,
                    spill_readers,
                    spill_writers,
                },
            );
        }

        // launch one async task per *input* partition
        let mut spawned_tasks = Vec::with_capacity(num_input_partitions);
        for (i, (stream, metrics)) in
            std::mem::take(streams_and_metrics).into_iter().enumerate()
        {
            let txs: HashMap<_, _> = channels
                .iter()
                .map(|(partition, channels)| {
                    // In preserve_order mode: each input gets its own spill writer (index i)
                    // In non-preserve-order mode: all inputs share spill writer 0 via clone
                    let spill_writer_idx = if preserve_order { i } else { 0 };
                    (
                        *partition,
                        OutputChannel {
                            sender: channels.tx[i].clone(),
                            reservation: Arc::clone(&channels.reservation),
                            spill_writer: channels.spill_writers[spill_writer_idx]
                                .clone(),
                        },
                    )
                })
                .collect();

            // Extract senders for wait_for_task before moving txs
            let senders: HashMap<_, _> = txs
                .iter()
                .map(|(partition, channel)| (*partition, channel.sender.clone()))
                .collect();

            let input_task = SpawnedTask::spawn(RepartitionExec::pull_from_input(
                stream,
                txs,
                partitioning.clone(),
                metrics,
                // preserve_order depends on partition index to start from 0
                if preserve_order { 0 } else { i },
                num_input_partitions,
            ));

            // In a separate task, wait for each input to be done
            // (and pass along any errors, including panic!s)
            let wait_for_task =
                SpawnedTask::spawn(RepartitionExec::wait_for_task(input_task, senders));
            spawned_tasks.push(wait_for_task);
        }
        *self = Self::ConsumingInputStreams(ConsumingInputStreamsState {
            channels,
            abort_helper: Arc::new(spawned_tasks),
        });
        match self {
            RepartitionExecState::ConsumingInputStreams(value) => Ok(value),
            _ => unreachable!(),
        }
    }
}

/// A utility that can be used to partition batches based on [`Partitioning`]
pub struct BatchPartitioner {
    state: BatchPartitionerState,
    timer: metrics::Time,
}

enum BatchPartitionerState {
    Hash {
        exprs: Vec<Arc<dyn PhysicalExpr>>,
        num_partitions: usize,
        hash_buffer: Vec<u64>,
        indices: Vec<Vec<u32>>,
    },
    RoundRobin {
        num_partitions: usize,
        next_idx: usize,
    },
}

/// Fixed RandomState used for hash repartitioning to ensure consistent behavior across
/// executions and runs.
pub const REPARTITION_RANDOM_STATE: SeededRandomState =
    SeededRandomState::with_seeds(0, 0, 0, 0);

impl BatchPartitioner {
    /// Create a new [`BatchPartitioner`] for hash-based repartitioning.
    ///
    /// # Parameters
    /// - `exprs`: Expressions used to compute the hash for each input row.
    /// - `num_partitions`: Total number of output partitions.
    /// - `timer`: Metric used to record time spent during repartitioning.
    ///
    /// # Notes
    /// This constructor cannot fail and performs no validation.
    pub fn new_hash_partitioner(
        exprs: Vec<Arc<dyn PhysicalExpr>>,
        num_partitions: usize,
        timer: metrics::Time,
    ) -> Self {
        Self {
            state: BatchPartitionerState::Hash {
                exprs,
                num_partitions,
                hash_buffer: vec![],
                indices: vec![vec![]; num_partitions],
            },
            timer,
        }
    }

    /// Create a new [`BatchPartitioner`] for round-robin repartitioning.
    ///
    /// # Parameters
    /// - `num_partitions`: Total number of output partitions.
    /// - `timer`: Metric used to record time spent during repartitioning.
    /// - `input_partition`: Index of the current input partition.
    /// - `num_input_partitions`: Total number of input partitions.
    ///
    /// # Notes
    /// The starting output partition is derived from the input partition
    /// to avoid skew when multiple input partitions are used.
    pub fn new_round_robin_partitioner(
        num_partitions: usize,
        timer: metrics::Time,
        input_partition: usize,
        num_input_partitions: usize,
    ) -> Self {
        Self {
            state: BatchPartitionerState::RoundRobin {
                num_partitions,
                next_idx: (input_partition * num_partitions) / num_input_partitions,
            },
            timer,
        }
    }
    /// Create a new [`BatchPartitioner`] based on the provided [`Partitioning`] scheme.
    ///
    /// This is a convenience constructor that delegates to the specialized
    /// hash or round-robin constructors depending on the partitioning variant.
    ///
    /// # Parameters
    /// - `partitioning`: Partitioning scheme to apply (hash or round-robin).
    /// - `timer`: Metric used to record time spent during repartitioning.
    /// - `input_partition`: Index of the current input partition.
    /// - `num_input_partitions`: Total number of input partitions.
    ///
    /// # Errors
    /// Returns an error if the provided partitioning scheme is not supported.
    pub fn try_new(
        partitioning: Partitioning,
        timer: metrics::Time,
        input_partition: usize,
        num_input_partitions: usize,
    ) -> Result<Self> {
        match partitioning {
            Partitioning::Hash(exprs, num_partitions) => {
                Ok(Self::new_hash_partitioner(exprs, num_partitions, timer))
            }
            Partitioning::RoundRobinBatch(num_partitions) => {
                Ok(Self::new_round_robin_partitioner(
                    num_partitions,
                    timer,
                    input_partition,
                    num_input_partitions,
                ))
            }
            other => {
                not_impl_err!("Unsupported repartitioning scheme {other:?}")
            }
        }
    }

    /// Partition the provided [`RecordBatch`] into one or more partitioned [`RecordBatch`]
    /// based on the [`Partitioning`] specified on construction
    ///
    /// `f` will be called for each partitioned [`RecordBatch`] with the corresponding
    /// partition index. Any error returned by `f` will be immediately returned by this
    /// function without attempting to publish further [`RecordBatch`]
    ///
    /// The time spent repartitioning, not including time spent in `f` will be recorded
    /// to the [`metrics::Time`] provided on construction
    pub fn partition<F>(&mut self, batch: RecordBatch, mut f: F) -> Result<()>
    where
        F: FnMut(usize, RecordBatch) -> Result<()>,
    {
        self.partition_iter(batch)?.try_for_each(|res| match res {
            Ok((partition, batch)) => f(partition, batch),
            Err(e) => Err(e),
        })
    }

    /// Actual implementation of [`partition`](Self::partition).
    ///
    /// The reason this was pulled out is that we need to have a variant of `partition` that works w/ sync functions,
    /// and one that works w/ async. Using an iterator as an intermediate representation was the best way to achieve
    /// this (so we don't need to clone the entire implementation).
    fn partition_iter(
        &mut self,
        batch: RecordBatch,
    ) -> Result<impl Iterator<Item = Result<(usize, RecordBatch)>> + Send + '_> {
        let it: Box<dyn Iterator<Item = Result<(usize, RecordBatch)>> + Send> =
            match &mut self.state {
                BatchPartitionerState::RoundRobin {
                    num_partitions,
                    next_idx,
                } => {
                    let idx = *next_idx;
                    *next_idx = (*next_idx + 1) % *num_partitions;
                    Box::new(std::iter::once(Ok((idx, batch))))
                }
                BatchPartitionerState::Hash {
                    exprs,
                    num_partitions: partitions,
                    hash_buffer,
                    indices,
                } => {
                    // Tracking time required for distributing indexes across output partitions
                    let timer = self.timer.timer();

                    let arrays =
                        evaluate_expressions_to_arrays(exprs.as_slice(), &batch)?;

                    hash_buffer.clear();
                    hash_buffer.resize(batch.num_rows(), 0);

                    create_hashes(
                        &arrays,
                        REPARTITION_RANDOM_STATE.random_state(),
                        hash_buffer,
                    )?;

                    indices.iter_mut().for_each(|v| v.clear());

                    for (index, hash) in hash_buffer.iter().enumerate() {
                        indices[(*hash % *partitions as u64) as usize].push(index as u32);
                    }

                    // Finished building index-arrays for output partitions
                    timer.done();

                    // Borrowing partitioner timer to prevent moving `self` to closure
                    let partitioner_timer = &self.timer;

                    let mut partitioned_batches = vec![];
                    for (partition, p_indices) in indices.iter_mut().enumerate() {
                        if !p_indices.is_empty() {
                            let taken_indices = std::mem::take(p_indices);
                            let indices_array: PrimitiveArray<UInt32Type> =
                                taken_indices.into();

                            // Tracking time required for repartitioned batches construction
                            let _timer = partitioner_timer.timer();

                            // Produce batches based on indices
                            let columns =
                                take_arrays(batch.columns(), &indices_array, None)?;

                            let mut options = RecordBatchOptions::new();
                            options = options.with_row_count(Some(indices_array.len()));
                            let batch = RecordBatch::try_new_with_options(
                                batch.schema(),
                                columns,
                                &options,
                            )
                            .unwrap();

                            partitioned_batches.push(Ok((partition, batch)));

                            // Return the taken vec
                            let (_, buffer, _) = indices_array.into_parts();
                            let mut vec =
                                buffer.into_inner().into_vec::<u32>().map_err(|e| {
                                    internal_datafusion_err!(
                                        "Could not convert buffer to vec: {e:?}"
                                    )
                                })?;
                            vec.clear();
                            *p_indices = vec;
                        }
                    }

                    Box::new(partitioned_batches.into_iter())
                }
            };

        Ok(it)
    }

    // return the number of output partitions
    fn num_partitions(&self) -> usize {
        match self.state {
            BatchPartitionerState::RoundRobin { num_partitions, .. } => num_partitions,
            BatchPartitionerState::Hash { num_partitions, .. } => num_partitions,
        }
    }
}

/// Maps `N` input partitions to `M` output partitions based on a
/// [`Partitioning`] scheme.
///
/// # Background
///
/// DataFusion, like most other commercial systems, with the
/// notable exception of DuckDB, uses the "Exchange Operator" based
/// approach to parallelism which works well in practice given
/// sufficient care in implementation.
///
/// DataFusion's planner picks the target number of partitions and
/// then [`RepartitionExec`] redistributes [`RecordBatch`]es to that number
/// of output partitions.
///
/// For example, given `target_partitions=3` (trying to use 3 cores)
/// but scanning an input with 2 partitions, `RepartitionExec` can be
/// used to get 3 even streams of `RecordBatch`es
///
///
///```text
///        ▲                  ▲                  ▲
///        │                  │                  │
///        │                  │                  │
///        │                  │                  │
/// ┌───────────────┐  ┌───────────────┐  ┌───────────────┐
/// │    GroupBy    │  │    GroupBy    │  │    GroupBy    │
/// │   (Partial)   │  │   (Partial)   │  │   (Partial)   │
/// └───────────────┘  └───────────────┘  └───────────────┘
///        ▲                  ▲                  ▲
///        └──────────────────┼──────────────────┘
//////              ┌─────────────────────────┐
///              │     RepartitionExec     │
///              │   (hash/round robin)    │
///              └─────────────────────────┘
///                         ▲   ▲
///             ┌───────────┘   └───────────┐
///             │                           │
///             │                           │
///        .─────────.                 .─────────.
///     ,─'           '─.           ,─'           '─.
///    ;      Input      :         ;      Input      :
///    :   Partition 0   ;         :   Partition 1   ;
///     ╲               ╱           ╲               ╱
///      '─.         ,─'             '─.         ,─'
///         `───────'                   `───────'
/// ```
///
/// # Error Handling
///
/// If any of the input partitions return an error, the error is propagated to
/// all output partitions and inputs are not polled again.
///
/// # Output Ordering
///
/// If more than one stream is being repartitioned, the output will be some
/// arbitrary interleaving (and thus unordered) unless
/// [`Self::with_preserve_order`] specifies otherwise.
///
/// # Spilling Architecture
///
/// RepartitionExec uses [`SpillPool`](crate::spill::spill_pool) channels to handle
/// memory pressure during repartitioning. Each (input partition, output partition)
/// pair gets its own SpillPool channel for FIFO ordering.
///
/// ```text
/// Input Partitions (N)          Output Partitions (M)
/// ────────────────────          ─────────────────────
///
///    Input 0 ──┐                      ┌──▶ Output 0
///              │  ┌──────────────┐    │
///              ├─▶│ SpillPool    │────┤
///              │  │ [In0→Out0]   │    │
///    Input 1 ──┤  └──────────────┘    ├──▶ Output 1
///              │                       │
///              │  ┌──────────────┐    │
///              ├─▶│ SpillPool    │────┤
///              │  │ [In1→Out0]   │    │
///    Input 2 ──┤  └──────────────┘    ├──▶ Output 2
///              │                      │
///              │       ... (N×M SpillPools total)
///              │                      │
///              │  ┌──────────────┐    │
///              └─▶│ SpillPool    │────┘
///                 │ [InN→OutM]   │
///                 └──────────────┘
///
/// Each SpillPool maintains FIFO order for its (input, output) pair.
/// See `RepartitionBatch` for details on the memory/spill decision logic.
/// ```
///
/// # Footnote
///
/// The "Exchange Operator" was first described in the 1989 paper
/// [Encapsulation of parallelism in the Volcano query processing
/// system Paper](https://dl.acm.org/doi/pdf/10.1145/93605.98720)
/// which uses the term "Exchange" for the concept of repartitioning
/// data across threads.
///
/// For more background, please also see the [Optimizing Repartitions in DataFusion] blog.
///
/// [Optimizing Repartitions in DataFusion]: https://datafusion.apache.org/blog/2025/12/15/avoid-consecutive-repartitions
#[derive(Debug, Clone)]
pub struct RepartitionExec {
    /// Input execution plan
    input: Arc<dyn ExecutionPlan>,
    /// Inner state that is initialized when the parent calls .execute() on this node
    /// and consumed as soon as the parent starts consuming this node.
    state: Arc<Mutex<RepartitionExecState>>,
    /// Execution metrics
    metrics: ExecutionPlanMetricsSet,
    /// Boolean flag to decide whether to preserve ordering. If true means
    /// `SortPreservingRepartitionExec`, false means `RepartitionExec`.
    preserve_order: bool,
    /// Cache holding plan properties like equivalences, output partitioning etc.
    cache: Arc<PlanProperties>,
}

#[derive(Debug, Clone)]
struct RepartitionMetrics {
    /// Time in nanos to execute child operator and fetch batches
    fetch_time: metrics::Time,
    /// Repartitioning elapsed time in nanos
    repartition_time: metrics::Time,
    /// Time in nanos for sending resulting batches to channels.
    ///
    /// One metric per output partition.
    send_time: Vec<metrics::Time>,
}

impl RepartitionMetrics {
    pub fn new(
        input_partition: usize,
        num_output_partitions: usize,
        metrics: &ExecutionPlanMetricsSet,
    ) -> Self {
        // Time in nanos to execute child operator and fetch batches
        let fetch_time =
            MetricBuilder::new(metrics).subset_time("fetch_time", input_partition);

        // Time in nanos to perform repartitioning
        let repartition_time =
            MetricBuilder::new(metrics).subset_time("repartition_time", input_partition);

        // Time in nanos for sending resulting batches to channels
        let send_time = (0..num_output_partitions)
            .map(|output_partition| {
                let label =
                    metrics::Label::new("outputPartition", output_partition.to_string());
                MetricBuilder::new(metrics)
                    .with_label(label)
                    .subset_time("send_time", input_partition)
            })
            .collect();

        Self {
            fetch_time,
            repartition_time,
            send_time,
        }
    }
}

impl RepartitionExec {
    /// Input execution plan
    pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
        &self.input
    }

    /// Partitioning scheme to use
    pub fn partitioning(&self) -> &Partitioning {
        &self.cache.partitioning
    }

    /// Get preserve_order flag of the RepartitionExec
    /// `true` means `SortPreservingRepartitionExec`, `false` means `RepartitionExec`
    pub fn preserve_order(&self) -> bool {
        self.preserve_order
    }

    /// Get name used to display this Exec
    pub fn name(&self) -> &str {
        "RepartitionExec"
    }

    fn with_new_children_and_same_properties(
        &self,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Self {
        Self {
            input: children.swap_remove(0),
            metrics: ExecutionPlanMetricsSet::new(),
            state: Default::default(),
            ..Self::clone(self)
        }
    }
}

impl DisplayAs for RepartitionExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result {
        let input_partition_count = self.input.output_partitioning().partition_count();
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(
                    f,
                    "{}: partitioning={}, input_partitions={}",
                    self.name(),
                    self.partitioning(),
                    input_partition_count,
                )?;

                if self.preserve_order {
                    write!(f, ", preserve_order=true")?;
                } else if input_partition_count <= 1
                    && self.input.output_ordering().is_some()
                {
                    // Make it explicit that repartition maintains sortedness for a single input partition even
                    // when `preserve_sort order` is false
                    write!(f, ", maintains_sort_order=true")?;
                }

                if let Some(sort_exprs) = self.sort_exprs() {
                    write!(f, ", sort_exprs={}", sort_exprs.clone())?;
                }
                Ok(())
            }
            DisplayFormatType::TreeRender => {
                writeln!(f, "partitioning_scheme={}", self.partitioning(),)?;
                let output_partition_count = self.partitioning().partition_count();
                let input_to_output_partition_str =
                    format!("{input_partition_count} -> {output_partition_count}");
                writeln!(
                    f,
                    "partition_count(in->out)={input_to_output_partition_str}"
                )?;

                if self.preserve_order {
                    writeln!(f, "preserve_order={}", self.preserve_order)?;
                }
                Ok(())
            }
        }
    }
}

impl ExecutionPlan for RepartitionExec {
    fn name(&self) -> &'static str {
        "RepartitionExec"
    }

    /// Return a reference to Any that can be used for downcasting
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn properties(&self) -> &Arc<PlanProperties> {
        &self.cache
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        vec![&self.input]
    }

    fn with_new_children(
        self: Arc<Self>,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        check_if_same_properties!(self, children);
        let mut repartition = RepartitionExec::try_new(
            children.swap_remove(0),
            self.partitioning().clone(),
        )?;
        if self.preserve_order {
            repartition = repartition.with_preserve_order();
        }
        Ok(Arc::new(repartition))
    }

    fn benefits_from_input_partitioning(&self) -> Vec<bool> {
        vec![matches!(self.partitioning(), Partitioning::Hash(_, _))]
    }

    fn maintains_input_order(&self) -> Vec<bool> {
        Self::maintains_input_order_helper(self.input(), self.preserve_order)
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        trace!(
            "Start {}::execute for partition: {}",
            self.name(),
            partition
        );

        let spill_metrics = SpillMetrics::new(&self.metrics, partition);

        let input = Arc::clone(&self.input);
        let partitioning = self.partitioning().clone();
        let metrics = self.metrics.clone();
        let preserve_order = self.sort_exprs().is_some();
        let name = self.name().to_owned();
        let schema = self.schema();
        let schema_captured = Arc::clone(&schema);

        let spill_manager = SpillManager::new(
            Arc::clone(&context.runtime_env()),
            spill_metrics,
            input.schema(),
        );

        // Get existing ordering to use for merging
        let sort_exprs = self.sort_exprs().cloned();

        let state = Arc::clone(&self.state);
        if let Some(mut state) = state.try_lock() {
            state.ensure_input_streams_initialized(
                &input,
                &metrics,
                partitioning.partition_count(),
                &context,
            )?;
        }

        let num_input_partitions = input.output_partitioning().partition_count();

        let stream = futures::stream::once(async move {
            // lock scope
            let (rx, reservation, spill_readers, abort_helper) = {
                // lock mutexes
                let mut state = state.lock();
                let state = state.consume_input_streams(
                    &input,
                    &metrics,
                    &partitioning,
                    preserve_order,
                    &name,
                    &context,
                    spill_manager.clone(),
                )?;

                // now return stream for the specified *output* partition which will
                // read from the channel
                let PartitionChannels {
                    rx,
                    reservation,
                    spill_readers,
                    ..
                } = state
                    .channels
                    .remove(&partition)
                    .expect("partition not used yet");

                (
                    rx,
                    reservation,
                    spill_readers,
                    Arc::clone(&state.abort_helper),
                )
            };

            trace!(
                "Before returning stream in {name}::execute for partition: {partition}"
            );

            if preserve_order {
                // Store streams from all the input partitions:
                // Each input partition gets its own spill reader to maintain proper FIFO ordering
                let input_streams = rx
                    .into_iter()
                    .zip(spill_readers)
                    .map(|(receiver, spill_stream)| {
                        // In preserve_order mode, each receiver corresponds to exactly one input partition
                        Box::pin(PerPartitionStream::new(
                            Arc::clone(&schema_captured),
                            receiver,
                            Arc::clone(&abort_helper),
                            Arc::clone(&reservation),
                            spill_stream,
                            1, // Each receiver handles one input partition
                            BaselineMetrics::new(&metrics, partition),
                            None, // subsequent merge sort already does batching https://github.com/apache/datafusion/blob/e4dcf0c85611ad0bd291f03a8e03fe56d773eb16/datafusion/physical-plan/src/sorts/merge.rs#L286
                        )) as SendableRecordBatchStream
                    })
                    .collect::<Vec<_>>();
                // Note that receiver size (`rx.len()`) and `num_input_partitions` are same.

                // Merge streams (while preserving ordering) coming from
                // input partitions to this partition:
                let fetch = None;
                let merge_reservation =
                    MemoryConsumer::new(format!("{name}[Merge {partition}]"))
                        .register(context.memory_pool());
                StreamingMergeBuilder::new()
                    .with_streams(input_streams)
                    .with_schema(schema_captured)
                    .with_expressions(&sort_exprs.unwrap())
                    .with_metrics(BaselineMetrics::new(&metrics, partition))
                    .with_batch_size(context.session_config().batch_size())
                    .with_fetch(fetch)
                    .with_reservation(merge_reservation)
                    .with_spill_manager(spill_manager)
                    .build()
            } else {
                // Non-preserve-order case: single input stream, so use the first spill reader
                let spill_stream = spill_readers
                    .into_iter()
                    .next()
                    .expect("at least one spill reader should exist");

                Ok(Box::pin(PerPartitionStream::new(
                    schema_captured,
                    rx.into_iter()
                        .next()
                        .expect("at least one receiver should exist"),
                    abort_helper,
                    reservation,
                    spill_stream,
                    num_input_partitions,
                    BaselineMetrics::new(&metrics, partition),
                    Some(context.session_config().batch_size()),
                )) as SendableRecordBatchStream)
            }
        })
        .try_flatten();
        let stream = RecordBatchStreamAdapter::new(schema, stream);
        Ok(Box::pin(stream))
    }

    fn metrics(&self) -> Option<MetricsSet> {
        Some(self.metrics.clone_inner())
    }

    fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
        if let Some(partition) = partition {
            let partition_count = self.partitioning().partition_count();
            if partition_count == 0 {
                return Ok(Statistics::new_unknown(&self.schema()));
            }

            assert_or_internal_err!(
                partition < partition_count,
                "RepartitionExec invalid partition {} (expected less than {})",
                partition,
                partition_count
            );

            let mut stats = self.input.partition_statistics(None)?;

            // Distribute statistics across partitions
            stats.num_rows = stats
                .num_rows
                .get_value()
                .map(|rows| Precision::Inexact(rows / partition_count))
                .unwrap_or(Precision::Absent);
            stats.total_byte_size = stats
                .total_byte_size
                .get_value()
                .map(|bytes| Precision::Inexact(bytes / partition_count))
                .unwrap_or(Precision::Absent);

            // Make all column stats unknown
            stats.column_statistics = stats
                .column_statistics
                .iter()
                .map(|_| ColumnStatistics::new_unknown())
                .collect();

            Ok(stats)
        } else {
            self.input.partition_statistics(None)
        }
    }

    fn cardinality_effect(&self) -> CardinalityEffect {
        CardinalityEffect::Equal
    }

    fn try_swapping_with_projection(
        &self,
        projection: &ProjectionExec,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        // If the projection does not narrow the schema, we should not try to push it down.
        if projection.expr().len() >= projection.input().schema().fields().len() {
            return Ok(None);
        }

        // If pushdown is not beneficial or applicable, break it.
        if projection.benefits_from_input_partitioning()[0]
            || !all_columns(projection.expr())
        {
            return Ok(None);
        }

        let new_projection = make_with_child(projection, self.input())?;

        let new_partitioning = match self.partitioning() {
            Partitioning::Hash(partitions, size) => {
                let mut new_partitions = vec![];
                for partition in partitions {
                    let Some(new_partition) =
                        update_expr(partition, projection.expr(), false)?
                    else {
                        return Ok(None);
                    };
                    new_partitions.push(new_partition);
                }
                Partitioning::Hash(new_partitions, *size)
            }
            others => others.clone(),
        };

        Ok(Some(Arc::new(RepartitionExec::try_new(
            new_projection,
            new_partitioning,
        )?)))
    }

    fn gather_filters_for_pushdown(
        &self,
        _phase: FilterPushdownPhase,
        parent_filters: Vec<Arc<dyn PhysicalExpr>>,
        _config: &ConfigOptions,
    ) -> Result<FilterDescription> {
        FilterDescription::from_children(parent_filters, &self.children())
    }

    fn handle_child_pushdown_result(
        &self,
        _phase: FilterPushdownPhase,
        child_pushdown_result: ChildPushdownResult,
        _config: &ConfigOptions,
    ) -> Result<FilterPushdownPropagation<Arc<dyn ExecutionPlan>>> {
        Ok(FilterPushdownPropagation::if_all(child_pushdown_result))
    }

    fn try_pushdown_sort(
        &self,
        order: &[PhysicalSortExpr],
    ) -> Result<SortOrderPushdownResult<Arc<dyn ExecutionPlan>>> {
        // RepartitionExec only maintains input order if preserve_order is set
        // or if there's only one partition
        if !self.maintains_input_order()[0] {
            return Ok(SortOrderPushdownResult::Unsupported);
        }

        // Delegate to the child and wrap with a new RepartitionExec
        self.input.try_pushdown_sort(order)?.try_map(|new_input| {
            let mut new_repartition =
                RepartitionExec::try_new(new_input, self.partitioning().clone())?;
            if self.preserve_order {
                new_repartition = new_repartition.with_preserve_order();
            }
            Ok(Arc::new(new_repartition) as Arc<dyn ExecutionPlan>)
        })
    }

    fn repartitioned(
        &self,
        target_partitions: usize,
        _config: &ConfigOptions,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        use Partitioning::*;
        let mut new_properties = PlanProperties::clone(&self.cache);
        new_properties.partitioning = match new_properties.partitioning {
            RoundRobinBatch(_) => RoundRobinBatch(target_partitions),
            Hash(hash, _) => Hash(hash, target_partitions),
            UnknownPartitioning(_) => UnknownPartitioning(target_partitions),
        };
        Ok(Some(Arc::new(Self {
            input: Arc::clone(&self.input),
            state: Arc::clone(&self.state),
            metrics: self.metrics.clone(),
            preserve_order: self.preserve_order,
            cache: new_properties.into(),
        })))
    }
}

impl RepartitionExec {
    /// Create a new RepartitionExec, that produces output `partitioning`, and
    /// does not preserve the order of the input (see [`Self::with_preserve_order`]
    /// for more details)
    pub fn try_new(
        input: Arc<dyn ExecutionPlan>,
        partitioning: Partitioning,
    ) -> Result<Self> {
        let preserve_order = false;
        let cache = Self::compute_properties(&input, partitioning, preserve_order);
        Ok(RepartitionExec {
            input,
            state: Default::default(),
            metrics: ExecutionPlanMetricsSet::new(),
            preserve_order,
            cache: Arc::new(cache),
        })
    }

    fn maintains_input_order_helper(
        input: &Arc<dyn ExecutionPlan>,
        preserve_order: bool,
    ) -> Vec<bool> {
        // We preserve ordering when repartition is order preserving variant or input partitioning is 1
        vec![preserve_order || input.output_partitioning().partition_count() <= 1]
    }

    fn eq_properties_helper(
        input: &Arc<dyn ExecutionPlan>,
        preserve_order: bool,
    ) -> EquivalenceProperties {
        // Equivalence Properties
        let mut eq_properties = input.equivalence_properties().clone();
        // If the ordering is lost, reset the ordering equivalence class:
        if !Self::maintains_input_order_helper(input, preserve_order)[0] {
            eq_properties.clear_orderings();
        }
        // When there are more than one input partitions, they will be fused at the output.
        // Therefore, remove per partition constants.
        if input.output_partitioning().partition_count() > 1 {
            eq_properties.clear_per_partition_constants();
        }
        eq_properties
    }

    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
    fn compute_properties(
        input: &Arc<dyn ExecutionPlan>,
        partitioning: Partitioning,
        preserve_order: bool,
    ) -> PlanProperties {
        PlanProperties::new(
            Self::eq_properties_helper(input, preserve_order),
            partitioning,
            input.pipeline_behavior(),
            input.boundedness(),
        )
        .with_scheduling_type(SchedulingType::Cooperative)
        .with_evaluation_type(EvaluationType::Eager)
    }

    /// Specify if this repartitioning operation should preserve the order of
    /// rows from its input when producing output. Preserving order is more
    /// expensive at runtime, so should only be set if the output of this
    /// operator can take advantage of it.
    ///
    /// If the input is not ordered, or has only one partition, this is a no op,
    /// and the node remains a `RepartitionExec`.
    pub fn with_preserve_order(mut self) -> Self {
        self.preserve_order =
                // If the input isn't ordered, there is no ordering to preserve
                self.input.output_ordering().is_some() &&
                // if there is only one input partition, merging is not required
                // to maintain order
                self.input.output_partitioning().partition_count() > 1;
        let eq_properties = Self::eq_properties_helper(&self.input, self.preserve_order);
        Arc::make_mut(&mut self.cache).set_eq_properties(eq_properties);
        self
    }

    /// Return the sort expressions that are used to merge
    fn sort_exprs(&self) -> Option<&LexOrdering> {
        if self.preserve_order {
            self.input.output_ordering()
        } else {
            None
        }
    }

    /// Pulls data from the specified input plan, feeding it to the
    /// output partitions based on the desired partitioning
    ///
    /// `output_channels` holds the output sending channels for each output partition
    async fn pull_from_input(
        mut stream: SendableRecordBatchStream,
        mut output_channels: HashMap<usize, OutputChannel>,
        partitioning: Partitioning,
        metrics: RepartitionMetrics,
        input_partition: usize,
        num_input_partitions: usize,
    ) -> Result<()> {
        let mut partitioner = match &partitioning {
            Partitioning::Hash(exprs, num_partitions) => {
                BatchPartitioner::new_hash_partitioner(
                    exprs.clone(),
                    *num_partitions,
                    metrics.repartition_time.clone(),
                )
            }
            Partitioning::RoundRobinBatch(num_partitions) => {
                BatchPartitioner::new_round_robin_partitioner(
                    *num_partitions,
                    metrics.repartition_time.clone(),
                    input_partition,
                    num_input_partitions,
                )
            }
            other => {
                return not_impl_err!("Unsupported repartitioning scheme {other:?}");
            }
        };

        // While there are still outputs to send to, keep pulling inputs
        let mut batches_until_yield = partitioner.num_partitions();
        while !output_channels.is_empty() {
            // fetch the next batch
            let timer = metrics.fetch_time.timer();
            let result = stream.next().await;
            timer.done();

            // Input is done
            let batch = match result {
                Some(result) => result?,
                None => break,
            };

            // Handle empty batch
            if batch.num_rows() == 0 {
                continue;
            }

            for res in partitioner.partition_iter(batch)? {
                let (partition, batch) = res?;
                let size = batch.get_array_memory_size();

                let timer = metrics.send_time[partition].timer();
                // if there is still a receiver, send to it
                if let Some(channel) = output_channels.get_mut(&partition) {
                    let (batch_to_send, is_memory_batch) =
                        match channel.reservation.lock().try_grow(size) {
                            Ok(_) => {
                                // Memory available - send in-memory batch
                                (RepartitionBatch::Memory(batch), true)
                            }
                            Err(_) => {
                                // We're memory limited - spill to SpillPool
                                // SpillPool handles file handle reuse and rotation
                                channel.spill_writer.push_batch(&batch)?;
                                // Send marker indicating batch was spilled
                                (RepartitionBatch::Spilled, false)
                            }
                        };

                    if channel.sender.send(Some(Ok(batch_to_send))).await.is_err() {
                        // If the other end has hung up, it was an early shutdown (e.g. LIMIT)
                        // Only shrink memory if it was a memory batch
                        if is_memory_batch {
                            channel.reservation.lock().shrink(size);
                        }
                        output_channels.remove(&partition);
                    }
                }
                timer.done();
            }

            // If the input stream is endless, we may spin forever and
            // never yield back to tokio.  See
            // https://github.com/apache/datafusion/issues/5278.
            //
            // However, yielding on every batch causes a bottleneck
            // when running with multiple cores. See
            // https://github.com/apache/datafusion/issues/6290
            //
            // Thus, heuristically yield after producing num_partition
            // batches
            //
            // In round robin this is ideal as each input will get a
            // new batch. In hash partitioning it may yield too often
            // on uneven distributions even if some partition can not
            // make progress, but parallelism is going to be limited
            // in that case anyways
            if batches_until_yield == 0 {
                tokio::task::yield_now().await;
                batches_until_yield = partitioner.num_partitions();
            } else {
                batches_until_yield -= 1;
            }
        }

        // Spill writers will auto-finalize when dropped
        // No need for explicit flush
        Ok(())
    }

    /// Waits for `input_task` which is consuming one of the inputs to
    /// complete. Upon each successful completion, sends a `None` to
    /// each of the output tx channels to signal one of the inputs is
    /// complete. Upon error, propagates the errors to all output tx
    /// channels.
    async fn wait_for_task(
        input_task: SpawnedTask<Result<()>>,
        txs: HashMap<usize, DistributionSender<MaybeBatch>>,
    ) {
        // wait for completion, and propagate error
        // note we ignore errors on send (.ok) as that means the receiver has already shutdown.

        match input_task.join().await {
            // Error in joining task
            Err(e) => {
                let e = Arc::new(e);

                for (_, tx) in txs {
                    let err = Err(DataFusionError::Context(
                        "Join Error".to_string(),
                        Box::new(DataFusionError::External(Box::new(Arc::clone(&e)))),
                    ));
                    tx.send(Some(err)).await.ok();
                }
            }
            // Error from running input task
            Ok(Err(e)) => {
                // send the same Arc'd error to all output partitions
                let e = Arc::new(e);

                for (_, tx) in txs {
                    // wrap it because need to send error to all output partitions
                    let err = Err(DataFusionError::from(&e));
                    tx.send(Some(err)).await.ok();
                }
            }
            // Input task completed successfully
            Ok(Ok(())) => {
                // notify each output partition that this input partition has no more data
                for (_partition, tx) in txs {
                    tx.send(None).await.ok();
                }
            }
        }
    }
}

/// State for tracking whether we're reading from memory channel or spill stream.
///
/// This state machine ensures proper ordering when batches are mixed between memory
/// and spilled storage. When a [`RepartitionBatch::Spilled`] marker is received,
/// the stream must block on the spill stream until the corresponding batch arrives.
///
/// # State Machine
///
/// ```text
///                        ┌─────────────────┐
///                   ┌───▶│  ReadingMemory  │◀───┐
///                   │    └────────┬────────┘    │
///                   │             │             │
///                   │     Poll channel          │
///                   │             │             │
///                   │  ┌──────────┼─────────────┐
///                   │  │          │             │
///                   │  ▼          ▼             │
///                   │ Memory   Spilled          │
///       Got batch   │ batch    marker           │
///       from spill  │  │          │             │
///                   │  │          ▼             │
///                   │  │  ┌──────────────────┐  │
///                   │  │  │ ReadingSpilled   │  │
///                   │  │  └────────┬─────────┘  │
///                   │  │           │            │
///                   │  │   Poll spill_stream    │
///                   │  │           │            │
///                   │  │           ▼            │
///                   │  │      Get batch         │
///                   │  │           │            │
///                   └──┴───────────┴────────────┘
/////////                           Return batch
///                     (Order preserved within
///                      (input, output) pair)
/// ```
///
/// The transition to `ReadingSpilled` blocks further channel polling to maintain
/// FIFO ordering - we cannot read the next item from the channel until the spill
/// stream provides the current batch.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
enum StreamState {
    /// Reading from the memory channel (normal operation)
    ReadingMemory,
    /// Waiting for a spilled batch from the spill stream.
    /// Must not poll channel until spilled batch is received to preserve ordering.
    ReadingSpilled,
}

/// This struct converts a receiver to a stream.
/// Receiver receives data on an SPSC channel.
struct PerPartitionStream {
    /// Schema wrapped by Arc
    schema: SchemaRef,

    /// channel containing the repartitioned batches
    receiver: DistributionReceiver<MaybeBatch>,

    /// Handle to ensure background tasks are killed when no longer needed.
    _drop_helper: Arc<Vec<SpawnedTask<()>>>,

    /// Memory reservation.
    reservation: SharedMemoryReservation,

    /// Infinite stream for reading from the spill pool
    spill_stream: SendableRecordBatchStream,

    /// Internal state indicating if we are reading from memory or spill stream
    state: StreamState,

    /// Number of input partitions that have not yet finished.
    /// In non-preserve-order mode, multiple input partitions send to the same channel,
    /// each sending None when complete. We must wait for all of them.
    remaining_partitions: usize,

    /// Execution metrics
    baseline_metrics: BaselineMetrics,

    /// None for sort preserving variant (merge sort already does coalescing)
    batch_coalescer: Option<LimitedBatchCoalescer>,
}

impl PerPartitionStream {
    #[expect(clippy::too_many_arguments)]
    fn new(
        schema: SchemaRef,
        receiver: DistributionReceiver<MaybeBatch>,
        drop_helper: Arc<Vec<SpawnedTask<()>>>,
        reservation: SharedMemoryReservation,
        spill_stream: SendableRecordBatchStream,
        num_input_partitions: usize,
        baseline_metrics: BaselineMetrics,
        batch_size: Option<usize>,
    ) -> Self {
        let batch_coalescer =
            batch_size.map(|s| LimitedBatchCoalescer::new(Arc::clone(&schema), s, None));
        Self {
            schema,
            receiver,
            _drop_helper: drop_helper,
            reservation,
            spill_stream,
            state: StreamState::ReadingMemory,
            remaining_partitions: num_input_partitions,
            baseline_metrics,
            batch_coalescer,
        }
    }

    fn poll_next_inner(
        self: &mut Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<Option<Result<RecordBatch>>> {
        use futures::StreamExt;
        let cloned_time = self.baseline_metrics.elapsed_compute().clone();
        let _timer = cloned_time.timer();

        loop {
            match self.state {
                StreamState::ReadingMemory => {
                    // Poll the memory channel for next message
                    let value = match self.receiver.recv().poll_unpin(cx) {
                        Poll::Ready(v) => v,
                        Poll::Pending => {
                            // Nothing from channel, wait
                            return Poll::Pending;
                        }
                    };

                    match value {
                        Some(Some(v)) => match v {
                            Ok(RepartitionBatch::Memory(batch)) => {
                                // Release memory and return batch
                                self.reservation
                                    .lock()
                                    .shrink(batch.get_array_memory_size());
                                return Poll::Ready(Some(Ok(batch)));
                            }
                            Ok(RepartitionBatch::Spilled) => {
                                // Batch was spilled, transition to reading from spill stream
                                // We must block on spill stream until we get the batch
                                // to preserve ordering
                                self.state = StreamState::ReadingSpilled;
                                continue;
                            }
                            Err(e) => {
                                return Poll::Ready(Some(Err(e)));
                            }
                        },
                        Some(None) => {
                            // One input partition finished
                            self.remaining_partitions -= 1;
                            if self.remaining_partitions == 0 {
                                // All input partitions finished
                                return Poll::Ready(None);
                            }
                            // Continue to poll for more data from other partitions
                            continue;
                        }
                        None => {
                            // Channel closed unexpectedly
                            return Poll::Ready(None);
                        }
                    }
                }
                StreamState::ReadingSpilled => {
                    // Poll spill stream for the spilled batch
                    match self.spill_stream.poll_next_unpin(cx) {
                        Poll::Ready(Some(Ok(batch))) => {
                            self.state = StreamState::ReadingMemory;
                            return Poll::Ready(Some(Ok(batch)));
                        }
                        Poll::Ready(Some(Err(e))) => {
                            return Poll::Ready(Some(Err(e)));
                        }
                        Poll::Ready(None) => {
                            // Spill stream ended, keep draining the memory channel
                            self.state = StreamState::ReadingMemory;
                        }
                        Poll::Pending => {
                            // Spilled batch not ready yet, must wait
                            // This preserves ordering by blocking until spill data arrives
                            return Poll::Pending;
                        }
                    }
                }
            }
        }
    }

    fn poll_next_and_coalesce(
        self: &mut Pin<&mut Self>,
        cx: &mut Context<'_>,
        coalescer: &mut LimitedBatchCoalescer,
    ) -> Poll<Option<Result<RecordBatch>>> {
        let cloned_time = self.baseline_metrics.elapsed_compute().clone();
        let mut completed = false;

        loop {
            if let Some(batch) = coalescer.next_completed_batch() {
                return Poll::Ready(Some(Ok(batch)));
            }
            if completed {
                return Poll::Ready(None);
            }

            match ready!(self.poll_next_inner(cx)) {
                Some(Ok(batch)) => {
                    let _timer = cloned_time.timer();
                    if let Err(err) = coalescer.push_batch(batch) {
                        return Poll::Ready(Some(Err(err)));
                    }
                }
                Some(err) => {
                    return Poll::Ready(Some(err));
                }
                None => {
                    completed = true;
                    let _timer = cloned_time.timer();
                    if let Err(err) = coalescer.finish() {
                        return Poll::Ready(Some(Err(err)));
                    }
                }
            }
        }
    }
}

impl Stream for PerPartitionStream {
    type Item = Result<RecordBatch>;

    fn poll_next(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<Option<Self::Item>> {
        let poll;
        if let Some(mut coalescer) = self.batch_coalescer.take() {
            poll = self.poll_next_and_coalesce(cx, &mut coalescer);
            self.batch_coalescer = Some(coalescer);
        } else {
            poll = self.poll_next_inner(cx);
        }
        self.baseline_metrics.record_poll(poll)
    }
}

impl RecordBatchStream for PerPartitionStream {
    /// Get the schema
    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }
}

#[cfg(test)]
mod tests {
    use std::collections::HashSet;

    use super::*;
    use crate::test::TestMemoryExec;
    use crate::{
        test::{
            assert_is_pending,
            exec::{
                BarrierExec, BlockingExec, ErrorExec, MockExec,
                assert_strong_count_converges_to_zero,
            },
        },
        {collect, expressions::col},
    };

    use arrow::array::{ArrayRef, StringArray, UInt32Array};
    use arrow::datatypes::{DataType, Field, Schema};
    use datafusion_common::cast::as_string_array;
    use datafusion_common::exec_err;
    use datafusion_common::test_util::batches_to_sort_string;
    use datafusion_common_runtime::JoinSet;
    use datafusion_execution::config::SessionConfig;
    use datafusion_execution::runtime_env::RuntimeEnvBuilder;
    use insta::assert_snapshot;

    #[tokio::test]
    async fn one_to_many_round_robin() -> Result<()> {
        // define input partitions
        let schema = test_schema();
        let partition = create_vec_batches(50);
        let partitions = vec![partition];

        // repartition from 1 input to 4 output
        let output_partitions =
            repartition(&schema, partitions, Partitioning::RoundRobinBatch(4)).await?;

        assert_eq!(4, output_partitions.len());
        for partition in &output_partitions {
            assert_eq!(1, partition.len());
        }
        assert_eq!(13 * 8, output_partitions[0][0].num_rows());
        assert_eq!(13 * 8, output_partitions[1][0].num_rows());
        assert_eq!(12 * 8, output_partitions[2][0].num_rows());
        assert_eq!(12 * 8, output_partitions[3][0].num_rows());

        Ok(())
    }

    #[tokio::test]
    async fn many_to_one_round_robin() -> Result<()> {
        // define input partitions
        let schema = test_schema();
        let partition = create_vec_batches(50);
        let partitions = vec![partition.clone(), partition.clone(), partition.clone()];

        // repartition from 3 input to 1 output
        let output_partitions =
            repartition(&schema, partitions, Partitioning::RoundRobinBatch(1)).await?;

        assert_eq!(1, output_partitions.len());
        assert_eq!(150 * 8, output_partitions[0][0].num_rows());

        Ok(())
    }

    #[tokio::test]
    async fn many_to_many_round_robin() -> Result<()> {
        // define input partitions
        let schema = test_schema();
        let partition = create_vec_batches(50);
        let partitions = vec![partition.clone(), partition.clone(), partition.clone()];

        // repartition from 3 input to 5 output
        let output_partitions =
            repartition(&schema, partitions, Partitioning::RoundRobinBatch(5)).await?;

        let total_rows_per_partition = 8 * 50 * 3 / 5;
        assert_eq!(5, output_partitions.len());
        for partition in output_partitions {
            assert_eq!(1, partition.len());
            assert_eq!(total_rows_per_partition, partition[0].num_rows());
        }

        Ok(())
    }

    #[tokio::test]
    async fn many_to_many_hash_partition() -> Result<()> {
        // define input partitions
        let schema = test_schema();
        let partition = create_vec_batches(50);
        let partitions = vec![partition.clone(), partition.clone(), partition.clone()];

        let output_partitions = repartition(
            &schema,
            partitions,
            Partitioning::Hash(vec![col("c0", &schema)?], 8),
        )
        .await?;

        let total_rows: usize = output_partitions
            .iter()
            .map(|x| x.iter().map(|x| x.num_rows()).sum::<usize>())
            .sum();

        assert_eq!(8, output_partitions.len());
        assert_eq!(total_rows, 8 * 50 * 3);

        Ok(())
    }

    #[tokio::test]
    async fn test_repartition_with_coalescing() -> Result<()> {
        let schema = test_schema();
        // create 50 batches, each having 8 rows
        let partition = create_vec_batches(50);
        let partitions = vec![partition.clone(), partition.clone()];
        let partitioning = Partitioning::RoundRobinBatch(1);

        let session_config = SessionConfig::new().with_batch_size(200);
        let task_ctx = TaskContext::default().with_session_config(session_config);
        let task_ctx = Arc::new(task_ctx);

        // create physical plan
        let exec = TestMemoryExec::try_new_exec(&partitions, Arc::clone(&schema), None)?;
        let exec = RepartitionExec::try_new(exec, partitioning)?;

        for i in 0..exec.partitioning().partition_count() {
            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
            while let Some(result) = stream.next().await {
                let batch = result?;
                assert_eq!(200, batch.num_rows());
            }
        }
        Ok(())
    }

    fn test_schema() -> Arc<Schema> {
        Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)]))
    }

    async fn repartition(
        schema: &SchemaRef,
        input_partitions: Vec<Vec<RecordBatch>>,
        partitioning: Partitioning,
    ) -> Result<Vec<Vec<RecordBatch>>> {
        let task_ctx = Arc::new(TaskContext::default());
        // create physical plan
        let exec =
            TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(schema), None)?;
        let exec = RepartitionExec::try_new(exec, partitioning)?;

        // execute and collect results
        let mut output_partitions = vec![];
        for i in 0..exec.partitioning().partition_count() {
            // execute this *output* partition and collect all batches
            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
            let mut batches = vec![];
            while let Some(result) = stream.next().await {
                batches.push(result?);
            }
            output_partitions.push(batches);
        }
        Ok(output_partitions)
    }

    #[tokio::test]
    async fn many_to_many_round_robin_within_tokio_task() -> Result<()> {
        let handle: SpawnedTask<Result<Vec<Vec<RecordBatch>>>> =
            SpawnedTask::spawn(async move {
                // define input partitions
                let schema = test_schema();
                let partition = create_vec_batches(50);
                let partitions =
                    vec![partition.clone(), partition.clone(), partition.clone()];

                // repartition from 3 input to 5 output
                repartition(&schema, partitions, Partitioning::RoundRobinBatch(5)).await
            });

        let output_partitions = handle.join().await.unwrap().unwrap();

        let total_rows_per_partition = 8 * 50 * 3 / 5;
        assert_eq!(5, output_partitions.len());
        for partition in output_partitions {
            assert_eq!(1, partition.len());
            assert_eq!(total_rows_per_partition, partition[0].num_rows());
        }

        Ok(())
    }

    #[tokio::test]
    async fn unsupported_partitioning() {
        let task_ctx = Arc::new(TaskContext::default());
        // have to send at least one batch through to provoke error
        let batch = RecordBatch::try_from_iter(vec![(
            "my_awesome_field",
            Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef,
        )])
        .unwrap();

        let schema = batch.schema();
        let input = MockExec::new(vec![Ok(batch)], schema);
        // This generates an error (partitioning type not supported)
        // but only after the plan is executed. The error should be
        // returned and no results produced
        let partitioning = Partitioning::UnknownPartitioning(1);
        let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap();
        let output_stream = exec.execute(0, task_ctx).unwrap();

        // Expect that an error is returned
        let result_string = crate::common::collect(output_stream)
            .await
            .unwrap_err()
            .to_string();
        assert!(
            result_string
                .contains("Unsupported repartitioning scheme UnknownPartitioning(1)"),
            "actual: {result_string}"
        );
    }

    #[tokio::test]
    async fn error_for_input_exec() {
        // This generates an error on a call to execute. The error
        // should be returned and no results produced.

        let task_ctx = Arc::new(TaskContext::default());
        let input = ErrorExec::new();
        let partitioning = Partitioning::RoundRobinBatch(1);
        let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap();

        // Expect that an error is returned
        let result_string = exec.execute(0, task_ctx).err().unwrap().to_string();

        assert!(
            result_string.contains("ErrorExec, unsurprisingly, errored in partition 0"),
            "actual: {result_string}"
        );
    }

    #[tokio::test]
    async fn repartition_with_error_in_stream() {
        let task_ctx = Arc::new(TaskContext::default());
        let batch = RecordBatch::try_from_iter(vec![(
            "my_awesome_field",
            Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef,
        )])
        .unwrap();

        // input stream returns one good batch and then one error. The
        // error should be returned.
        let err = exec_err!("bad data error");

        let schema = batch.schema();
        let input = MockExec::new(vec![Ok(batch), err], schema);
        let partitioning = Partitioning::RoundRobinBatch(1);
        let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap();

        // Note: this should pass (the stream can be created) but the
        // error when the input is executed should get passed back
        let output_stream = exec.execute(0, task_ctx).unwrap();

        // Expect that an error is returned
        let result_string = crate::common::collect(output_stream)
            .await
            .unwrap_err()
            .to_string();
        assert!(
            result_string.contains("bad data error"),
            "actual: {result_string}"
        );
    }

    #[tokio::test]
    async fn repartition_with_delayed_stream() {
        let task_ctx = Arc::new(TaskContext::default());
        let batch1 = RecordBatch::try_from_iter(vec![(
            "my_awesome_field",
            Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef,
        )])
        .unwrap();

        let batch2 = RecordBatch::try_from_iter(vec![(
            "my_awesome_field",
            Arc::new(StringArray::from(vec!["frob", "baz"])) as ArrayRef,
        )])
        .unwrap();

        // The mock exec doesn't return immediately (instead it
        // requires the input to wait at least once)
        let schema = batch1.schema();
        let expected_batches = vec![batch1.clone(), batch2.clone()];
        let input = MockExec::new(vec![Ok(batch1), Ok(batch2)], schema);
        let partitioning = Partitioning::RoundRobinBatch(1);

        let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap();

        assert_snapshot!(batches_to_sort_string(&expected_batches), @r"
        +------------------+
        | my_awesome_field |
        +------------------+
        | bar              |
        | baz              |
        | foo              |
        | frob             |
        +------------------+
        ");

        let output_stream = exec.execute(0, task_ctx).unwrap();
        let batches = crate::common::collect(output_stream).await.unwrap();

        assert_snapshot!(batches_to_sort_string(&batches), @r"
        +------------------+
        | my_awesome_field |
        +------------------+
        | bar              |
        | baz              |
        | foo              |
        | frob             |
        +------------------+
        ");
    }

    #[tokio::test]
    async fn robin_repartition_with_dropping_output_stream() {
        let task_ctx = Arc::new(TaskContext::default());
        let partitioning = Partitioning::RoundRobinBatch(2);
        // The barrier exec waits to be pinged
        // requires the input to wait at least once)
        let input = Arc::new(make_barrier_exec());

        // partition into two output streams
        let exec = RepartitionExec::try_new(
            Arc::clone(&input) as Arc<dyn ExecutionPlan>,
            partitioning,
        )
        .unwrap();

        let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap();
        let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap();

        // now, purposely drop output stream 0
        // *before* any outputs are produced
        drop(output_stream0);

        // Now, start sending input
        let mut background_task = JoinSet::new();
        background_task.spawn(async move {
            input.wait().await;
        });

        // output stream 1 should *not* error and have one of the input batches
        let batches = crate::common::collect(output_stream1).await.unwrap();

        assert_snapshot!(batches_to_sort_string(&batches), @r"
        +------------------+
        | my_awesome_field |
        +------------------+
        | baz              |
        | frob             |
        | gar              |
        | goo              |
        +------------------+
        ");
    }

    #[tokio::test]
    // As the hash results might be different on different platforms or
    // with different compilers, we will compare the same execution with
    // and without dropping the output stream.
    async fn hash_repartition_with_dropping_output_stream() {
        let task_ctx = Arc::new(TaskContext::default());
        let partitioning = Partitioning::Hash(
            vec![Arc::new(crate::expressions::Column::new(
                "my_awesome_field",
                0,
            ))],
            2,
        );

        // We first collect the results without dropping the output stream.
        let input = Arc::new(make_barrier_exec());
        let exec = RepartitionExec::try_new(
            Arc::clone(&input) as Arc<dyn ExecutionPlan>,
            partitioning.clone(),
        )
        .unwrap();
        let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap();
        let mut background_task = JoinSet::new();
        background_task.spawn(async move {
            input.wait().await;
        });
        let batches_without_drop = crate::common::collect(output_stream1).await.unwrap();

        // run some checks on the result
        let items_vec = str_batches_to_vec(&batches_without_drop);
        let items_set: HashSet<&str> = items_vec.iter().copied().collect();
        assert_eq!(items_vec.len(), items_set.len());
        let source_str_set: HashSet<&str> =
            ["foo", "bar", "frob", "baz", "goo", "gar", "grob", "gaz"]
                .iter()
                .copied()
                .collect();
        assert_eq!(items_set.difference(&source_str_set).count(), 0);

        // Now do the same but dropping the stream before waiting for the barrier
        let input = Arc::new(make_barrier_exec());
        let exec = RepartitionExec::try_new(
            Arc::clone(&input) as Arc<dyn ExecutionPlan>,
            partitioning,
        )
        .unwrap();
        let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap();
        let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap();
        // now, purposely drop output stream 0
        // *before* any outputs are produced
        drop(output_stream0);
        let mut background_task = JoinSet::new();
        background_task.spawn(async move {
            input.wait().await;
        });
        let batches_with_drop = crate::common::collect(output_stream1).await.unwrap();

        let items_vec_with_drop = str_batches_to_vec(&batches_with_drop);
        let items_set_with_drop: HashSet<&str> =
            items_vec_with_drop.iter().copied().collect();
        assert_eq!(
            items_set_with_drop.symmetric_difference(&items_set).count(),
            0
        );
    }

    fn str_batches_to_vec(batches: &[RecordBatch]) -> Vec<&str> {
        batches
            .iter()
            .flat_map(|batch| {
                assert_eq!(batch.columns().len(), 1);
                let string_array = as_string_array(batch.column(0))
                    .expect("Unexpected type for repartitioned batch");

                string_array
                    .iter()
                    .map(|v| v.expect("Unexpected null"))
                    .collect::<Vec<_>>()
            })
            .collect::<Vec<_>>()
    }

    /// Create a BarrierExec that returns two partitions of two batches each
    fn make_barrier_exec() -> BarrierExec {
        let batch1 = RecordBatch::try_from_iter(vec![(
            "my_awesome_field",
            Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef,
        )])
        .unwrap();

        let batch2 = RecordBatch::try_from_iter(vec![(
            "my_awesome_field",
            Arc::new(StringArray::from(vec!["frob", "baz"])) as ArrayRef,
        )])
        .unwrap();

        let batch3 = RecordBatch::try_from_iter(vec![(
            "my_awesome_field",
            Arc::new(StringArray::from(vec!["goo", "gar"])) as ArrayRef,
        )])
        .unwrap();

        let batch4 = RecordBatch::try_from_iter(vec![(
            "my_awesome_field",
            Arc::new(StringArray::from(vec!["grob", "gaz"])) as ArrayRef,
        )])
        .unwrap();

        // The barrier exec waits to be pinged
        // requires the input to wait at least once)
        let schema = batch1.schema();
        BarrierExec::new(vec![vec![batch1, batch2], vec![batch3, batch4]], schema)
    }

    #[tokio::test]
    async fn test_drop_cancel() -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());
        let schema =
            Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)]));

        let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 2));
        let refs = blocking_exec.refs();
        let repartition_exec = Arc::new(RepartitionExec::try_new(
            blocking_exec,
            Partitioning::UnknownPartitioning(1),
        )?);

        let fut = collect(repartition_exec, task_ctx);
        let mut fut = fut.boxed();

        assert_is_pending(&mut fut);
        drop(fut);
        assert_strong_count_converges_to_zero(refs).await;

        Ok(())
    }

    #[tokio::test]
    async fn hash_repartition_avoid_empty_batch() -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());
        let batch = RecordBatch::try_from_iter(vec![(
            "a",
            Arc::new(StringArray::from(vec!["foo"])) as ArrayRef,
        )])
        .unwrap();
        let partitioning = Partitioning::Hash(
            vec![Arc::new(crate::expressions::Column::new("a", 0))],
            2,
        );
        let schema = batch.schema();
        let input = MockExec::new(vec![Ok(batch)], schema);
        let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap();
        let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap();
        let batch0 = crate::common::collect(output_stream0).await.unwrap();
        let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap();
        let batch1 = crate::common::collect(output_stream1).await.unwrap();
        assert!(batch0.is_empty() || batch1.is_empty());
        Ok(())
    }

    #[tokio::test]
    async fn repartition_with_spilling() -> Result<()> {
        // Test that repartition successfully spills to disk when memory is constrained
        let schema = test_schema();
        let partition = create_vec_batches(50);
        let input_partitions = vec![partition];
        let partitioning = Partitioning::RoundRobinBatch(4);

        // Set up context with very tight memory limit to force spilling
        let runtime = RuntimeEnvBuilder::default()
            .with_memory_limit(1, 1.0)
            .build_arc()?;

        let task_ctx = TaskContext::default().with_runtime(runtime);
        let task_ctx = Arc::new(task_ctx);

        // create physical plan
        let exec =
            TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?;
        let exec = RepartitionExec::try_new(exec, partitioning)?;

        // Collect all partitions - should succeed by spilling to disk
        let mut total_rows = 0;
        for i in 0..exec.partitioning().partition_count() {
            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
            while let Some(result) = stream.next().await {
                let batch = result?;
                total_rows += batch.num_rows();
            }
        }

        // Verify we got all the data (50 batches * 8 rows each)
        assert_eq!(total_rows, 50 * 8);

        // Verify spilling metrics to confirm spilling actually happened
        let metrics = exec.metrics().unwrap();
        assert!(
            metrics.spill_count().unwrap() > 0,
            "Expected spill_count > 0, but got {:?}",
            metrics.spill_count()
        );
        println!("Spilled {} times", metrics.spill_count().unwrap());
        assert!(
            metrics.spilled_bytes().unwrap() > 0,
            "Expected spilled_bytes > 0, but got {:?}",
            metrics.spilled_bytes()
        );
        println!(
            "Spilled {} bytes in {} spills",
            metrics.spilled_bytes().unwrap(),
            metrics.spill_count().unwrap()
        );
        assert!(
            metrics.spilled_rows().unwrap() > 0,
            "Expected spilled_rows > 0, but got {:?}",
            metrics.spilled_rows()
        );
        println!("Spilled {} rows", metrics.spilled_rows().unwrap());

        Ok(())
    }

    #[tokio::test]
    async fn repartition_with_partial_spilling() -> Result<()> {
        // Test that repartition can handle partial spilling (some batches in memory, some spilled)
        let schema = test_schema();
        let partition = create_vec_batches(50);
        let input_partitions = vec![partition];
        let partitioning = Partitioning::RoundRobinBatch(4);

        // Set up context with moderate memory limit to force partial spilling
        // 2KB should allow some batches in memory but force others to spill
        let runtime = RuntimeEnvBuilder::default()
            .with_memory_limit(2 * 1024, 1.0)
            .build_arc()?;

        let task_ctx = TaskContext::default().with_runtime(runtime);
        let task_ctx = Arc::new(task_ctx);

        // create physical plan
        let exec =
            TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?;
        let exec = RepartitionExec::try_new(exec, partitioning)?;

        // Collect all partitions - should succeed with partial spilling
        let mut total_rows = 0;
        for i in 0..exec.partitioning().partition_count() {
            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
            while let Some(result) = stream.next().await {
                let batch = result?;
                total_rows += batch.num_rows();
            }
        }

        // Verify we got all the data (50 batches * 8 rows each)
        assert_eq!(total_rows, 50 * 8);

        // Verify partial spilling metrics
        let metrics = exec.metrics().unwrap();
        let spill_count = metrics.spill_count().unwrap();
        let spilled_rows = metrics.spilled_rows().unwrap();
        let spilled_bytes = metrics.spilled_bytes().unwrap();

        assert!(
            spill_count > 0,
            "Expected some spilling to occur, but got spill_count={spill_count}"
        );
        assert!(
            spilled_rows > 0 && spilled_rows < total_rows,
            "Expected partial spilling (0 < spilled_rows < {total_rows}), but got spilled_rows={spilled_rows}"
        );
        assert!(
            spilled_bytes > 0,
            "Expected some bytes to be spilled, but got spilled_bytes={spilled_bytes}"
        );

        println!(
            "Partial spilling: spilled {} out of {} rows ({:.1}%) in {} spills, {} bytes",
            spilled_rows,
            total_rows,
            (spilled_rows as f64 / total_rows as f64) * 100.0,
            spill_count,
            spilled_bytes
        );

        Ok(())
    }

    #[tokio::test]
    async fn repartition_without_spilling() -> Result<()> {
        // Test that repartition does not spill when there's ample memory
        let schema = test_schema();
        let partition = create_vec_batches(50);
        let input_partitions = vec![partition];
        let partitioning = Partitioning::RoundRobinBatch(4);

        // Set up context with generous memory limit - no spilling should occur
        let runtime = RuntimeEnvBuilder::default()
            .with_memory_limit(10 * 1024 * 1024, 1.0) // 10MB
            .build_arc()?;

        let task_ctx = TaskContext::default().with_runtime(runtime);
        let task_ctx = Arc::new(task_ctx);

        // create physical plan
        let exec =
            TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?;
        let exec = RepartitionExec::try_new(exec, partitioning)?;

        // Collect all partitions - should succeed without spilling
        let mut total_rows = 0;
        for i in 0..exec.partitioning().partition_count() {
            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
            while let Some(result) = stream.next().await {
                let batch = result?;
                total_rows += batch.num_rows();
            }
        }

        // Verify we got all the data (50 batches * 8 rows each)
        assert_eq!(total_rows, 50 * 8);

        // Verify no spilling occurred
        let metrics = exec.metrics().unwrap();
        assert_eq!(
            metrics.spill_count(),
            Some(0),
            "Expected no spilling, but got spill_count={:?}",
            metrics.spill_count()
        );
        assert_eq!(
            metrics.spilled_bytes(),
            Some(0),
            "Expected no bytes spilled, but got spilled_bytes={:?}",
            metrics.spilled_bytes()
        );
        assert_eq!(
            metrics.spilled_rows(),
            Some(0),
            "Expected no rows spilled, but got spilled_rows={:?}",
            metrics.spilled_rows()
        );

        println!("No spilling occurred - all data processed in memory");

        Ok(())
    }

    #[tokio::test]
    async fn oom() -> Result<()> {
        use datafusion_execution::disk_manager::{DiskManagerBuilder, DiskManagerMode};

        // Test that repartition fails with OOM when disk manager is disabled
        let schema = test_schema();
        let partition = create_vec_batches(50);
        let input_partitions = vec![partition];
        let partitioning = Partitioning::RoundRobinBatch(4);

        // Setup context with memory limit but NO disk manager (explicitly disabled)
        let runtime = RuntimeEnvBuilder::default()
            .with_memory_limit(1, 1.0)
            .with_disk_manager_builder(
                DiskManagerBuilder::default().with_mode(DiskManagerMode::Disabled),
            )
            .build_arc()?;

        let task_ctx = TaskContext::default().with_runtime(runtime);
        let task_ctx = Arc::new(task_ctx);

        // create physical plan
        let exec =
            TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?;
        let exec = RepartitionExec::try_new(exec, partitioning)?;

        // Attempt to execute - should fail with ResourcesExhausted error
        for i in 0..exec.partitioning().partition_count() {
            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
            let err = stream.next().await.unwrap().unwrap_err();
            let err = err.find_root();
            assert!(
                matches!(err, DataFusionError::ResourcesExhausted(_)),
                "Wrong error type: {err}",
            );
        }

        Ok(())
    }

    /// Create vector batches
    fn create_vec_batches(n: usize) -> Vec<RecordBatch> {
        let batch = create_batch();
        std::iter::repeat_n(batch, n).collect()
    }

    /// Create batch
    fn create_batch() -> RecordBatch {
        let schema = test_schema();
        RecordBatch::try_new(
            schema,
            vec![Arc::new(UInt32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8]))],
        )
        .unwrap()
    }

    /// Create batches with sequential values for ordering tests
    fn create_ordered_batches(num_batches: usize) -> Vec<RecordBatch> {
        let schema = test_schema();
        (0..num_batches)
            .map(|i| {
                let start = (i * 8) as u32;
                RecordBatch::try_new(
                    Arc::clone(&schema),
                    vec![Arc::new(UInt32Array::from(
                        (start..start + 8).collect::<Vec<_>>(),
                    ))],
                )
                .unwrap()
            })
            .collect()
    }

    #[tokio::test]
    async fn test_repartition_ordering_with_spilling() -> Result<()> {
        // Test that repartition preserves ordering when spilling occurs
        // This tests the state machine fix where we must block on spill_stream
        // when a Spilled marker is received, rather than continuing to poll the channel

        let schema = test_schema();
        // Create batches with sequential values: batch 0 has [0,1,2,3,4,5,6,7],
        // batch 1 has [8,9,10,11,12,13,14,15], etc.
        let partition = create_ordered_batches(20);
        let input_partitions = vec![partition];

        // Use RoundRobinBatch to ensure predictable ordering
        let partitioning = Partitioning::RoundRobinBatch(2);

        // Set up context with very tight memory limit to force spilling
        let runtime = RuntimeEnvBuilder::default()
            .with_memory_limit(1, 1.0)
            .build_arc()?;

        let task_ctx = TaskContext::default().with_runtime(runtime);
        let task_ctx = Arc::new(task_ctx);

        // create physical plan
        let exec =
            TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?;
        let exec = RepartitionExec::try_new(exec, partitioning)?;

        // Collect all output partitions
        let mut all_batches = Vec::new();
        for i in 0..exec.partitioning().partition_count() {
            let mut partition_batches = Vec::new();
            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
            while let Some(result) = stream.next().await {
                let batch = result?;
                partition_batches.push(batch);
            }
            all_batches.push(partition_batches);
        }

        // Verify spilling occurred
        let metrics = exec.metrics().unwrap();
        assert!(
            metrics.spill_count().unwrap() > 0,
            "Expected spilling to occur, but spill_count = 0"
        );

        // Verify ordering is preserved within each partition
        // With RoundRobinBatch, even batches go to partition 0, odd batches to partition 1
        for (partition_idx, batches) in all_batches.iter().enumerate() {
            let mut last_value = None;
            for batch in batches {
                let array = batch
                    .column(0)
                    .as_any()
                    .downcast_ref::<UInt32Array>()
                    .unwrap();

                for i in 0..array.len() {
                    let value = array.value(i);
                    if let Some(last) = last_value {
                        assert!(
                            value > last,
                            "Ordering violated in partition {partition_idx}: {value} is not greater than {last}"
                        );
                    }
                    last_value = Some(value);
                }
            }
        }

        Ok(())
    }
}

#[cfg(test)]
mod test {
    use arrow::array::record_batch;
    use arrow::compute::SortOptions;
    use arrow::datatypes::{DataType, Field, Schema};
    use datafusion_common::assert_batches_eq;

    use super::*;
    use crate::test::TestMemoryExec;
    use crate::union::UnionExec;

    use datafusion_physical_expr::expressions::col;
    use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr};

    /// Asserts that the plan is as expected
    ///
    /// `$EXPECTED_PLAN_LINES`: input plan
    /// `$PLAN`: the plan to optimized
    macro_rules! assert_plan {
        ($PLAN: expr,  @ $EXPECTED: expr) => {
            let formatted = crate::displayable($PLAN).indent(true).to_string();

            insta::assert_snapshot!(
                formatted,
                @$EXPECTED
            );
        };
    }

    #[tokio::test]
    async fn test_preserve_order() -> Result<()> {
        let schema = test_schema();
        let sort_exprs = sort_exprs(&schema);
        let source1 = sorted_memory_exec(&schema, sort_exprs.clone());
        let source2 = sorted_memory_exec(&schema, sort_exprs);
        // output has multiple partitions, and is sorted
        let union = UnionExec::try_new(vec![source1, source2])?;
        let exec = RepartitionExec::try_new(union, Partitioning::RoundRobinBatch(10))?
            .with_preserve_order();

        // Repartition should preserve order
        assert_plan!(&exec, @r"
        RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c0@0 ASC
          UnionExec
            DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC
            DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC
        ");
        Ok(())
    }

    #[tokio::test]
    async fn test_preserve_order_one_partition() -> Result<()> {
        let schema = test_schema();
        let sort_exprs = sort_exprs(&schema);
        let source = sorted_memory_exec(&schema, sort_exprs);
        // output is sorted, but has only a single partition, so no need to sort
        let exec = RepartitionExec::try_new(source, Partitioning::RoundRobinBatch(10))?
            .with_preserve_order();

        // Repartition should not preserve order
        assert_plan!(&exec, @r"
        RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1, maintains_sort_order=true
          DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC
        ");

        Ok(())
    }

    #[tokio::test]
    async fn test_preserve_order_input_not_sorted() -> Result<()> {
        let schema = test_schema();
        let source1 = memory_exec(&schema);
        let source2 = memory_exec(&schema);
        // output has multiple partitions, but is not sorted
        let union = UnionExec::try_new(vec![source1, source2])?;
        let exec = RepartitionExec::try_new(union, Partitioning::RoundRobinBatch(10))?
            .with_preserve_order();

        // Repartition should not preserve order, as there is no order to preserve
        assert_plan!(&exec, @r"
        RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2
          UnionExec
            DataSourceExec: partitions=1, partition_sizes=[0]
            DataSourceExec: partitions=1, partition_sizes=[0]
        ");
        Ok(())
    }

    #[tokio::test]
    async fn test_preserve_order_with_spilling() -> Result<()> {
        use datafusion_execution::TaskContext;
        use datafusion_execution::runtime_env::RuntimeEnvBuilder;

        // Create sorted input data across multiple partitions
        // Partition1: [1,3], [5,7], [9,11]
        // Partition2: [2,4], [6,8], [10,12]
        let batch1 = record_batch!(("c0", UInt32, [1, 3])).unwrap();
        let batch2 = record_batch!(("c0", UInt32, [2, 4])).unwrap();
        let batch3 = record_batch!(("c0", UInt32, [5, 7])).unwrap();
        let batch4 = record_batch!(("c0", UInt32, [6, 8])).unwrap();
        let batch5 = record_batch!(("c0", UInt32, [9, 11])).unwrap();
        let batch6 = record_batch!(("c0", UInt32, [10, 12])).unwrap();
        let schema = batch1.schema();
        let sort_exprs = LexOrdering::new([PhysicalSortExpr {
            expr: col("c0", &schema).unwrap(),
            options: SortOptions::default().asc(),
        }])
        .unwrap();
        let partition1 = vec![batch1.clone(), batch3.clone(), batch5.clone()];
        let partition2 = vec![batch2.clone(), batch4.clone(), batch6.clone()];
        let input_partitions = vec![partition1, partition2];

        // Set up context with tight memory limit to force spilling
        // Sorting needs some non-spillable memory, so 64 bytes should force spilling while still allowing the query to complete
        let runtime = RuntimeEnvBuilder::default()
            .with_memory_limit(64, 1.0)
            .build_arc()?;

        let task_ctx = TaskContext::default().with_runtime(runtime);
        let task_ctx = Arc::new(task_ctx);

        // Create physical plan with order preservation
        let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?
            .try_with_sort_information(vec![sort_exprs.clone(), sort_exprs])?;
        let exec = Arc::new(exec);
        let exec = Arc::new(TestMemoryExec::update_cache(&exec));
        // Repartition into 3 partitions with order preservation
        // We expect 1 batch per output partition after repartitioning
        let exec = RepartitionExec::try_new(exec, Partitioning::RoundRobinBatch(3))?
            .with_preserve_order();

        let mut batches = vec![];

        // Collect all partitions - should succeed by spilling to disk
        for i in 0..exec.partitioning().partition_count() {
            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
            while let Some(result) = stream.next().await {
                let batch = result?;
                batches.push(batch);
            }
        }

        #[rustfmt::skip]
        let expected = [
            [
                "+----+",
                "| c0 |",
                "+----+",
                "| 1  |",
                "| 2  |",
                "| 3  |",
                "| 4  |",
                "+----+",
            ],
            [
                "+----+",
                "| c0 |",
                "+----+",
                "| 5  |",
                "| 6  |",
                "| 7  |",
                "| 8  |",
                "+----+",
            ],
            [
                "+----+",
                "| c0 |",
                "+----+",
                "| 9  |",
                "| 10 |",
                "| 11 |",
                "| 12 |",
                "+----+",
            ],
        ];

        for (batch, expected) in batches.iter().zip(expected.iter()) {
            assert_batches_eq!(expected, std::slice::from_ref(batch));
        }

        // We should have spilled ~ all of the data.
        // - We spill data during the repartitioning phase
        // - We may also spill during the final merge sort
        let all_batches = [batch1, batch2, batch3, batch4, batch5, batch6];
        let metrics = exec.metrics().unwrap();
        assert!(
            metrics.spill_count().unwrap() > input_partitions.len(),
            "Expected spill_count > {} for order-preserving repartition, but got {:?}",
            input_partitions.len(),
            metrics.spill_count()
        );
        assert!(
            metrics.spilled_bytes().unwrap()
                > all_batches
                    .iter()
                    .map(|b| b.get_array_memory_size())
                    .sum::<usize>(),
            "Expected spilled_bytes > {} for order-preserving repartition, got {}",
            all_batches
                .iter()
                .map(|b| b.get_array_memory_size())
                .sum::<usize>(),
            metrics.spilled_bytes().unwrap()
        );
        assert!(
            metrics.spilled_rows().unwrap()
                >= all_batches.iter().map(|b| b.num_rows()).sum::<usize>(),
            "Expected spilled_rows > {} for order-preserving repartition, got {}",
            all_batches.iter().map(|b| b.num_rows()).sum::<usize>(),
            metrics.spilled_rows().unwrap()
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_hash_partitioning_with_spilling() -> Result<()> {
        use datafusion_execution::TaskContext;
        use datafusion_execution::runtime_env::RuntimeEnvBuilder;

        // Create input data similar to the round-robin test
        let batch1 = record_batch!(("c0", UInt32, [1, 3])).unwrap();
        let batch2 = record_batch!(("c0", UInt32, [2, 4])).unwrap();
        let batch3 = record_batch!(("c0", UInt32, [5, 7])).unwrap();
        let batch4 = record_batch!(("c0", UInt32, [6, 8])).unwrap();
        let schema = batch1.schema();

        let partition1 = vec![batch1.clone(), batch3.clone()];
        let partition2 = vec![batch2.clone(), batch4.clone()];
        let input_partitions = vec![partition1, partition2];

        // Set up context with memory limit to test hash partitioning with spilling infrastructure
        let runtime = RuntimeEnvBuilder::default()
            .with_memory_limit(1, 1.0)
            .build_arc()?;

        let task_ctx = TaskContext::default().with_runtime(runtime);
        let task_ctx = Arc::new(task_ctx);

        // Create physical plan with hash partitioning
        let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?;
        let exec = Arc::new(exec);
        let exec = Arc::new(TestMemoryExec::update_cache(&exec));
        // Hash partition into 2 partitions by column c0
        let hash_expr = col("c0", &schema)?;
        let exec =
            RepartitionExec::try_new(exec, Partitioning::Hash(vec![hash_expr], 2))?;

        // Collect all partitions concurrently using JoinSet - this prevents deadlock
        // where the distribution channel gate closes when all output channels are full
        let mut join_set = tokio::task::JoinSet::new();
        for i in 0..exec.partitioning().partition_count() {
            let stream = exec.execute(i, Arc::clone(&task_ctx))?;
            join_set.spawn(async move {
                let mut count = 0;
                futures::pin_mut!(stream);
                while let Some(result) = stream.next().await {
                    let batch = result?;
                    count += batch.num_rows();
                }
                Ok::<usize, DataFusionError>(count)
            });
        }

        // Wait for all partitions and sum the rows
        let mut total_rows = 0;
        while let Some(result) = join_set.join_next().await {
            total_rows += result.unwrap()?;
        }

        // Verify we got all rows back
        let all_batches = [batch1, batch2, batch3, batch4];
        let expected_rows: usize = all_batches.iter().map(|b| b.num_rows()).sum();
        assert_eq!(total_rows, expected_rows);

        // Verify metrics are available
        let metrics = exec.metrics().unwrap();
        // Just verify the metrics can be retrieved (spilling may or may not occur)
        let spill_count = metrics.spill_count().unwrap_or(0);
        assert!(spill_count > 0);
        let spilled_bytes = metrics.spilled_bytes().unwrap_or(0);
        assert!(spilled_bytes > 0);
        let spilled_rows = metrics.spilled_rows().unwrap_or(0);
        assert!(spilled_rows > 0);

        Ok(())
    }

    #[tokio::test]
    async fn test_repartition() -> Result<()> {
        let schema = test_schema();
        let sort_exprs = sort_exprs(&schema);
        let source = sorted_memory_exec(&schema, sort_exprs);
        // output is sorted, but has only a single partition, so no need to sort
        let exec = RepartitionExec::try_new(source, Partitioning::RoundRobinBatch(10))?
            .repartitioned(20, &Default::default())?
            .unwrap();

        // Repartition should not preserve order
        assert_plan!(exec.as_ref(), @r"
        RepartitionExec: partitioning=RoundRobinBatch(20), input_partitions=1, maintains_sort_order=true
          DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC
        ");
        Ok(())
    }

    fn test_schema() -> Arc<Schema> {
        Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)]))
    }

    fn sort_exprs(schema: &Schema) -> LexOrdering {
        [PhysicalSortExpr {
            expr: col("c0", schema).unwrap(),
            options: SortOptions::default(),
        }]
        .into()
    }

    fn memory_exec(schema: &SchemaRef) -> Arc<dyn ExecutionPlan> {
        TestMemoryExec::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap()
    }

    fn sorted_memory_exec(
        schema: &SchemaRef,
        sort_exprs: LexOrdering,
    ) -> Arc<dyn ExecutionPlan> {
        let exec = TestMemoryExec::try_new(&[vec![]], Arc::clone(schema), None)
            .unwrap()
            .try_with_sort_information(vec![sort_exprs])
            .unwrap();
        let exec = Arc::new(exec);
        Arc::new(TestMemoryExec::update_cache(&exec))
    }
}