datafusion-physical-plan 54.0.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
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258
3259
3260
3261
3262
3263
3264
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274
3275
3276
3277
3278
3279
3280
3281
3282
3283
3284
3285
3286
3287
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311
3312
3313
3314
3315
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363
3364
3365
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375
3376
3377
3378
3379
3380
3381
3382
3383
3384
3385
3386
3387
3388
3389
3390
3391
3392
3393
3394
3395
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405
3406
3407
3408
3409
3410
3411
3412
3413
3414
3415
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
3426
3427
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442
3443
3444
3445
3446
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458
3459
3460
3461
3462
3463
3464
3465
3466
3467
3468
3469
3470
3471
3472
3473
3474
3475
3476
3477
3478
3479
3480
3481
3482
3483
3484
3485
3486
3487
3488
3489
3490
3491
3492
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516
3517
3518
3519
3520
3521
3522
3523
3524
3525
3526
3527
3528
3529
3530
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548
3549
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599
3600
3601
3602
3603
3604
3605
3606
3607
3608
3609
3610
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636
3637
3638
3639
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651
3652
3653
3654
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691
3692
3693
3694
3695
3696
3697
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711
3712
3713
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745
3746
3747
3748
3749
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
3767
3768
3769
3770
3771
3772
3773
3774
3775
3776
3777
3778
3779
3780
3781
3782
3783
3784
3785
3786
3787
3788
3789
3790
3791
3792
3793
3794
3795
3796
3797
3798
3799
3800
3801
3802
3803
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818
3819
3820
3821
3822
3823
3824
3825
3826
3827
3828
3829
3830
3831
3832
3833
3834
3835
3836
3837
3838
3839
3840
3841
3842
3843
3844
3845
3846
3847
3848
3849
3850
3851
3852
3853
3854
3855
3856
3857
3858
3859
3860
3861
3862
3863
3864
3865
3866
3867
3868
3869
3870
3871
3872
3873
3874
3875
3876
3877
3878
3879
3880
3881
3882
3883
3884
3885
3886
3887
3888
3889
3890
3891
3892
3893
3894
3895
3896
3897
3898
3899
3900
3901
3902
3903
3904
3905
3906
3907
3908
3909
3910
3911
3912
3913
3914
3915
3916
3917
3918
3919
3920
3921
3922
3923
3924
3925
3926
3927
3928
3929
3930
3931
3932
// 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.

//! [`NestedLoopJoinExec`]: joins without equijoin (equality predicates).

use std::fmt::Formatter;
use std::ops::{BitOr, ControlFlow};
use std::sync::Arc;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::task::Poll;

use super::utils::{
    asymmetric_join_output_partitioning, need_produce_result_in_final,
    reorder_output_after_swap, swap_join_projection,
};
use crate::common::can_project;
use crate::execution_plan::{EmissionType, boundedness_from_children};
use crate::joins::SharedBitmapBuilder;
use crate::joins::utils::{
    BuildProbeJoinMetrics, ColumnIndex, JoinFilter, OnceAsync, OnceFut,
    build_join_schema, check_join_is_valid, estimate_join_statistics,
    need_produce_right_in_final,
};
use crate::metrics::{
    Count, ExecutionPlanMetricsSet, MetricBuilder, MetricType, MetricsSet, RatioMetrics,
};
use crate::projection::{
    EmbeddedProjection, JoinData, ProjectionExec, try_embed_projection,
    try_pushdown_through_join,
};
use crate::{
    DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties,
    PlanProperties, RecordBatchStream, SendableRecordBatchStream,
    check_if_same_properties,
};

use arrow::array::{
    Array, BooleanArray, BooleanBufferBuilder, RecordBatchOptions, UInt32Array,
    UInt64Array, new_null_array,
};
use arrow::buffer::BooleanBuffer;
use arrow::compute::{
    BatchCoalescer, concat_batches, filter, filter_record_batch, not, take,
};
use arrow::datatypes::{Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use arrow_schema::DataType;
use datafusion_common::cast::as_boolean_array;
use datafusion_common::{
    JoinSide, Result, ScalarValue, Statistics, arrow_err, assert_eq_or_internal_err,
    internal_datafusion_err, internal_err, project_schema, unwrap_or_internal_err,
};
use datafusion_execution::TaskContext;
use datafusion_execution::disk_manager::RefCountedTempFile;
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion_expr::JoinType;
use datafusion_physical_expr::equivalence::{
    ProjectionMapping, join_equivalence_properties,
};

use datafusion_physical_expr::projection::{ProjectionRef, combine_projections};
use futures::{Stream, StreamExt, TryStreamExt};
use log::debug;
use parking_lot::Mutex;

use crate::metrics::SpillMetrics;
use crate::spill::replayable_spill_input::ReplayableStreamSource;
use crate::spill::spill_manager::SpillManager;

#[expect(rustdoc::private_intra_doc_links)]
/// NestedLoopJoinExec is a build-probe join operator designed for joins that
/// do not have equijoin keys in their `ON` clause.
///
/// # Execution Flow
///
/// ```text
///                                                Incoming right batch
///                Left Side Buffered Batches
///                       ┌───────────┐              ┌───────────────┐
///                       │ ┌───────┐ │              │               │
///                       │ │       │ │              │               │
///  Current Left Row ───▶│ ├───────├─┤──────────┐   │               │
///                       │ │       │ │          │   └───────────────┘
///                       │ │       │ │          │           │
///                       │ │       │ │          │           │
///                       │ └───────┘ │          │           │
///                       │ ┌───────┐ │          │           │
///                       │ │       │ │          │     ┌─────┘
///                       │ │       │ │          │     │
///                       │ │       │ │          │     │
///                       │ │       │ │          │     │
///                       │ │       │ │          │     │
///                       │ └───────┘ │          ▼     ▼
///                       │   ......  │  ┌──────────────────────┐
///                       │           │  │X (Cartesian Product) │
///                       │           │  └──────────┬───────────┘
///                       └───────────┘             │
/////////                                      ┌───────┬───────────────┐
///                                      │       │               │
///                                      │       │               │
///                                      │       │               │
///                                      └───────┴───────────────┘
///                                        Intermediate Batch
///                                  (For join predicate evaluation)
/// ```
///
/// The execution follows a two-phase design:
///
/// ## 1. Buffering Left Input
/// - The operator eagerly buffers all left-side input batches into memory,
///   util a memory limit is reached.
///   Currently, an out-of-memory error will be thrown if all the left-side input batches
///   cannot fit into memory at once.
///   In the future, it's possible to make this case finish execution. (see
///   'Memory-limited Execution' section)
/// - The rationale for buffering the left side is that scanning the right side
///   can be expensive (e.g., decoding Parquet files), so buffering more left
///   rows reduces the number of right-side scan passes required.
///
/// ## 2. Probing Right Input
/// - Right-side input is streamed batch by batch.
/// - For each right-side batch:
///   - It evaluates the join filter against the full buffered left input.
///     This results in a Cartesian product between the right batch and each
///     left row -- with the join predicate/filter applied -- for each inner
///     loop iteration.
///   - Matched results are accumulated into an output buffer. (see more in
///     `Output Buffering Strategy` section)
/// - This process continues until all right-side input is consumed.
///
/// # Producing unmatched build-side data
/// - For special join types like left/full joins, it's required to also output
///   unmatched pairs. During execution, bitmaps are kept for both left and right
///   sides of the input; they'll be handled by dedicated states in `NLJStream`.
/// - The final output of the left side unmatched rows is handled by a single
///   partition for simplicity, since it only counts a small portion of the
///   execution time. (e.g. if probe side has 10k rows, the final output of
///   unmatched build side only roughly counts for 1/10k of the total time)
///
/// # Output Buffering Strategy
/// The operator uses an intermediate output buffer to accumulate results. Once
/// the output threshold is reached (currently set to the same value as
/// `batch_size` in the configuration), the results will be eagerly output.
///
/// # Extra Notes
/// - The operator always considers the **left** side as the build (buffered) side.
///   Therefore, the physical optimizer should assign the smaller input to the left.
/// - The design try to minimize the intermediate data size to approximately
///   1 batch, for better cache locality and memory efficiency.
///
/// # Memory-limited Execution
/// When the memory budget is exceeded during left-side buffering, the operator
/// falls back to a multi-pass strategy:
/// 1. Buffer as many left rows as fit in memory (one "chunk")
/// 2. On the first pass, the right side is both processed and spilled to disk
/// 3. For each subsequent left chunk, the right side is re-read from the spill file
///
/// The fallback is triggered automatically when the initial in-memory load
/// fails with `ResourcesExhausted` and disk spilling is available. Each
/// output partition independently re-executes the left child and manages
/// its own spill state.
///
/// All join types are supported. For RIGHT/FULL/RIGHT SEMI/RIGHT ANTI/
/// RIGHT MARK joins, a global right-side bitmap (indexed by right batch
/// sequence number) accumulates matches across all left chunks. After the
/// last left chunk is processed, the right side is replayed one more time
/// to emit unmatched right rows using the accumulated bitmap.
///
/// Tracking issue: <https://github.com/apache/datafusion/issues/15760>
///
/// # Clone / Shared State
/// Note this structure includes a [`OnceAsync`] that is used to coordinate the
/// loading of the left side with the processing in each output stream.
/// Therefore it can not be [`Clone`]
#[derive(Debug)]
pub struct NestedLoopJoinExec {
    /// left side
    pub(crate) left: Arc<dyn ExecutionPlan>,
    /// right side
    pub(crate) right: Arc<dyn ExecutionPlan>,
    /// Filters which are applied while finding matching rows
    pub(crate) filter: Option<JoinFilter>,
    /// How the join is performed
    pub(crate) join_type: JoinType,
    /// The full concatenated schema of left and right children should be distinct from
    /// the output schema of the operator
    join_schema: SchemaRef,
    /// Future that consumes left input and buffers it in memory
    ///
    /// This structure is *shared* across all output streams.
    ///
    /// Each output stream waits on the `OnceAsync` to signal the completion of
    /// the build(left) side data, and buffer them all for later joining.
    build_side_data: OnceAsync<JoinLeftData>,
    /// Shared left-side spill data for OOM fallback.
    ///
    /// When `build_side_data` fails with OOM, the first partition to
    /// initiate fallback spills the entire left side to disk. Other
    /// partitions share the same spill file via this `OnceAsync`,
    /// avoiding redundant re-execution of the left child.
    left_spill_data: Arc<OnceAsync<LeftSpillData>>,
    /// Information of index and left / right placement of columns
    column_indices: Vec<ColumnIndex>,
    /// Projection to apply to the output of the join
    projection: Option<ProjectionRef>,

    /// Execution metrics
    metrics: ExecutionPlanMetricsSet,
    /// Cache holding plan properties like equivalences, output partitioning etc.
    cache: Arc<PlanProperties>,
}

/// Helps to build [`NestedLoopJoinExec`].
pub struct NestedLoopJoinExecBuilder {
    left: Arc<dyn ExecutionPlan>,
    right: Arc<dyn ExecutionPlan>,
    join_type: JoinType,
    filter: Option<JoinFilter>,
    projection: Option<ProjectionRef>,
}

impl NestedLoopJoinExecBuilder {
    /// Make a new [`NestedLoopJoinExecBuilder`].
    pub fn new(
        left: Arc<dyn ExecutionPlan>,
        right: Arc<dyn ExecutionPlan>,
        join_type: JoinType,
    ) -> Self {
        Self {
            left,
            right,
            join_type,
            filter: None,
            projection: None,
        }
    }

    /// Set projection from the vector.
    pub fn with_projection(self, projection: Option<Vec<usize>>) -> Self {
        self.with_projection_ref(projection.map(Into::into))
    }

    /// Set projection from the shared reference.
    pub fn with_projection_ref(mut self, projection: Option<ProjectionRef>) -> Self {
        self.projection = projection;
        self
    }

    /// Set optional filter.
    pub fn with_filter(mut self, filter: Option<JoinFilter>) -> Self {
        self.filter = filter;
        self
    }

    /// Build resulting execution plan.
    pub fn build(self) -> Result<NestedLoopJoinExec> {
        let Self {
            left,
            right,
            join_type,
            filter,
            projection,
        } = self;

        let left_schema = left.schema();
        let right_schema = right.schema();
        check_join_is_valid(&left_schema, &right_schema, &[])?;
        let (join_schema, column_indices) =
            build_join_schema(&left_schema, &right_schema, &join_type);
        let join_schema = Arc::new(join_schema);
        let cache = NestedLoopJoinExec::compute_properties(
            &left,
            &right,
            &join_schema,
            join_type,
            projection.as_deref(),
        )?;
        Ok(NestedLoopJoinExec {
            left,
            right,
            filter,
            join_type,
            join_schema,
            build_side_data: Default::default(),
            left_spill_data: Arc::new(OnceAsync::default()),
            column_indices,
            projection,
            metrics: Default::default(),
            cache: Arc::new(cache),
        })
    }
}

impl From<&NestedLoopJoinExec> for NestedLoopJoinExecBuilder {
    fn from(exec: &NestedLoopJoinExec) -> Self {
        Self {
            left: Arc::clone(exec.left()),
            right: Arc::clone(exec.right()),
            join_type: exec.join_type,
            filter: exec.filter.clone(),
            projection: exec.projection.clone(),
        }
    }
}

impl NestedLoopJoinExec {
    /// Try to create a new [`NestedLoopJoinExec`]
    pub fn try_new(
        left: Arc<dyn ExecutionPlan>,
        right: Arc<dyn ExecutionPlan>,
        filter: Option<JoinFilter>,
        join_type: &JoinType,
        projection: Option<Vec<usize>>,
    ) -> Result<Self> {
        NestedLoopJoinExecBuilder::new(left, right, *join_type)
            .with_projection(projection)
            .with_filter(filter)
            .build()
    }

    /// left side
    pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
        &self.left
    }

    /// right side
    pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
        &self.right
    }

    /// Filters applied before join output
    pub fn filter(&self) -> Option<&JoinFilter> {
        self.filter.as_ref()
    }

    /// How the join is performed
    pub fn join_type(&self) -> &JoinType {
        &self.join_type
    }

    pub fn projection(&self) -> &Option<ProjectionRef> {
        &self.projection
    }

    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
    fn compute_properties(
        left: &Arc<dyn ExecutionPlan>,
        right: &Arc<dyn ExecutionPlan>,
        schema: &SchemaRef,
        join_type: JoinType,
        projection: Option<&[usize]>,
    ) -> Result<PlanProperties> {
        // Calculate equivalence properties:
        let mut eq_properties = join_equivalence_properties(
            left.equivalence_properties().clone(),
            right.equivalence_properties().clone(),
            &join_type,
            Arc::clone(schema),
            &Self::maintains_input_order(join_type),
            None,
            // No on columns in nested loop join
            &[],
        )?;

        let mut output_partitioning =
            asymmetric_join_output_partitioning(left, right, &join_type)?;

        let emission_type = if left.boundedness().is_unbounded() {
            EmissionType::Final
        } else if right.pipeline_behavior() == EmissionType::Incremental {
            match join_type {
                // If we only need to generate matched rows from the probe side,
                // we can emit rows incrementally.
                JoinType::Inner
                | JoinType::LeftSemi
                | JoinType::RightSemi
                | JoinType::Right
                | JoinType::RightAnti
                | JoinType::RightMark => EmissionType::Incremental,
                // If we need to generate unmatched rows from the *build side*,
                // we need to emit them at the end.
                JoinType::Left
                | JoinType::LeftAnti
                | JoinType::LeftMark
                | JoinType::Full => EmissionType::Both,
            }
        } else {
            right.pipeline_behavior()
        };

        if let Some(projection) = projection {
            // construct a map from the input expressions to the output expression of the Projection
            let projection_mapping = ProjectionMapping::from_indices(projection, schema)?;
            let out_schema = project_schema(schema, Some(&projection))?;
            output_partitioning =
                output_partitioning.project(&projection_mapping, &eq_properties);
            eq_properties = eq_properties.project(&projection_mapping, out_schema);
        }

        Ok(PlanProperties::new(
            eq_properties,
            output_partitioning,
            emission_type,
            boundedness_from_children([left, right]),
        ))
    }

    /// This join implementation does not preserve the input order of either side.
    fn maintains_input_order(_join_type: JoinType) -> Vec<bool> {
        vec![false, false]
    }

    pub fn contains_projection(&self) -> bool {
        self.projection.is_some()
    }

    pub fn with_projection(&self, projection: Option<Vec<usize>>) -> Result<Self> {
        let projection = projection.map(Into::into);
        // check if the projection is valid
        can_project(&self.schema(), projection.as_deref())?;
        let projection =
            combine_projections(projection.as_ref(), self.projection.as_ref())?;
        NestedLoopJoinExecBuilder::from(self)
            .with_projection_ref(projection)
            .build()
    }

    /// Returns a new `ExecutionPlan` that runs NestedLoopsJoins with the left
    /// and right inputs swapped.
    ///
    /// # Notes:
    ///
    /// This function should be called BEFORE inserting any repartitioning
    /// operators on the join's children. Check [`super::HashJoinExec::swap_inputs`]
    /// for more details.
    pub fn swap_inputs(&self) -> Result<Arc<dyn ExecutionPlan>> {
        let left = self.left();
        let right = self.right();
        let new_join = NestedLoopJoinExec::try_new(
            Arc::clone(right),
            Arc::clone(left),
            self.filter().map(JoinFilter::swap),
            &self.join_type().swap(),
            swap_join_projection(
                left.schema().fields().len(),
                right.schema().fields().len(),
                self.projection.as_deref(),
                self.join_type(),
            ),
        )?;

        // For Semi/Anti joins, swap result will produce same output schema,
        // no need to wrap them into additional projection
        let plan: Arc<dyn ExecutionPlan> = if matches!(
            self.join_type(),
            JoinType::LeftSemi
                | JoinType::RightSemi
                | JoinType::LeftAnti
                | JoinType::RightAnti
                | JoinType::LeftMark
                | JoinType::RightMark
        ) || self.projection.is_some()
        {
            Arc::new(new_join)
        } else {
            reorder_output_after_swap(
                Arc::new(new_join),
                &self.left().schema(),
                &self.right().schema(),
            )?
        };

        Ok(plan)
    }

    fn with_new_children_and_same_properties(
        &self,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Self {
        let left = children.swap_remove(0);
        let right = children.swap_remove(0);

        Self {
            left,
            right,
            metrics: ExecutionPlanMetricsSet::new(),
            build_side_data: Default::default(),
            left_spill_data: Arc::new(OnceAsync::default()),
            cache: Arc::clone(&self.cache),
            filter: self.filter.clone(),
            join_type: self.join_type,
            join_schema: Arc::clone(&self.join_schema),
            column_indices: self.column_indices.clone(),
            projection: self.projection.clone(),
        }
    }
}

impl DisplayAs for NestedLoopJoinExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                let display_filter = self.filter.as_ref().map_or_else(
                    || "".to_string(),
                    |f| format!(", filter={}", f.expression()),
                );
                let display_projections = if self.contains_projection() {
                    format!(
                        ", projection=[{}]",
                        self.projection
                            .as_ref()
                            .unwrap()
                            .iter()
                            .map(|index| format!(
                                "{}@{}",
                                self.join_schema.fields().get(*index).unwrap().name(),
                                index
                            ))
                            .collect::<Vec<_>>()
                            .join(", ")
                    )
                } else {
                    "".to_string()
                };
                write!(
                    f,
                    "NestedLoopJoinExec: join_type={:?}{}{}",
                    self.join_type, display_filter, display_projections
                )
            }
            DisplayFormatType::TreeRender => {
                if *self.join_type() != JoinType::Inner {
                    writeln!(f, "join_type={:?}", self.join_type)
                } else {
                    Ok(())
                }
            }
        }
    }
}

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

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

    fn required_input_distribution(&self) -> Vec<Distribution> {
        vec![
            Distribution::SinglePartition,
            Distribution::UnspecifiedDistribution,
        ]
    }

    fn maintains_input_order(&self) -> Vec<bool> {
        Self::maintains_input_order(self.join_type)
    }

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

    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        check_if_same_properties!(self, children);
        Ok(Arc::new(
            NestedLoopJoinExecBuilder::new(
                Arc::clone(&children[0]),
                Arc::clone(&children[1]),
                self.join_type,
            )
            .with_filter(self.filter.clone())
            .with_projection_ref(self.projection.clone())
            .build()?,
        ))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        assert_eq_or_internal_err!(
            self.left.output_partitioning().partition_count(),
            1,
            "Invalid NestedLoopJoinExec, the output partition count of the left child must be 1,\
                 consider using CoalescePartitionsExec or the EnforceDistribution rule"
        );

        let metrics = NestedLoopJoinMetrics::new(&self.metrics, partition);
        let batch_size = context.session_config().batch_size();

        // update column indices to reflect the projection
        let column_indices_after_projection = match self.projection.as_ref() {
            Some(projection) => projection
                .iter()
                .map(|i| self.column_indices[*i].clone())
                .collect(),
            None => self.column_indices.clone(),
        };

        let right_partition_count = self.right().output_partitioning().partition_count();

        // Always try to buffer all left data in memory via OnceFut.
        // If that fails with OOM, the stream will fallback to memory-limited
        // mode (if conditions allow).
        let load_reservation =
            MemoryConsumer::new(format!("NestedLoopJoinLoad[{partition}]"))
                .register(context.memory_pool());

        let build_side_data = self.build_side_data.try_once(|| {
            let stream = self.left.execute(0, Arc::clone(&context))?;

            Ok(collect_left_input(
                stream,
                metrics.join_metrics.clone(),
                load_reservation,
                need_produce_result_in_final(self.join_type),
                right_partition_count,
            ))
        })?;

        let probe_side_data = self.right.execute(partition, Arc::clone(&context))?;

        // Determine if OOM fallback to memory-limited mode is possible.
        // Conditions:
        // 1. Disk manager supports temp files (needed for spilling).
        // 2. FULL join with multiple right partitions is not yet supported
        //    in the fallback path. FULL join needs to track BOTH left-side
        //    matches (for unmatched left rows) AND right-side matches (for
        //    unmatched right rows). The fallback path builds a per-partition
        //    `JoinLeftData` with `probe_threads_counter == 1`, so each
        //    partition emits unmatched left rows based only on its own
        //    right-side matches, producing incorrect duplicate output for
        //    left rows that match in another partition. Other join types
        //    that need only one-sided final emission (LEFT, LEFT SEMI,
        //    LEFT ANTI, LEFT MARK) have a similar latent issue in the
        //    fallback path which predates this change; tracking is out of
        //    scope for this PR.
        let full_join_multi_partition =
            matches!(self.join_type, JoinType::Full) && right_partition_count > 1;
        let spill_state = if context.runtime_env().disk_manager.tmp_files_enabled()
            && !full_join_multi_partition
        {
            SpillState::Pending {
                left_plan: Arc::clone(&self.left),
                task_context: Arc::clone(&context),
                left_spill_data: Arc::clone(&self.left_spill_data),
            }
        } else {
            SpillState::Disabled
        };

        Ok(Box::pin(NestedLoopJoinStream::new(
            self.schema(),
            self.filter.clone(),
            self.join_type,
            probe_side_data,
            build_side_data,
            column_indices_after_projection,
            metrics,
            batch_size,
            spill_state,
        )))
    }

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

    fn partition_statistics(&self, partition: Option<usize>) -> Result<Arc<Statistics>> {
        // NestedLoopJoinExec is designed for joins without equijoin keys in the
        // ON clause (e.g., `t1 JOIN t2 ON (t1.v1 + t2.v1) % 2 = 0`). Any join
        // predicates are stored in `self.filter`, but `estimate_join_statistics`
        // currently doesn't support selectivity estimation for such arbitrary
        // filter expressions. We pass an empty join column list, which means
        // the cardinality estimation cannot use column statistics and returns
        // unknown row counts.
        let join_columns = Vec::new();

        // Left side is always a single partition (Distribution::SinglePartition),
        // so we always request overall stats with `None`. Right side can have
        // multiple partitions, so we forward the partition parameter to get
        // partition-specific statistics when requested.
        let left_stats = Arc::unwrap_or_clone(self.left.partition_statistics(None)?);
        let right_stats = Arc::unwrap_or_clone(match partition {
            Some(partition) => self.right.partition_statistics(Some(partition))?,
            None => self.right.partition_statistics(None)?,
        });

        let stats = estimate_join_statistics(
            left_stats,
            right_stats,
            &join_columns,
            &self.join_type,
            &self.join_schema,
        )?;

        Ok(Arc::new(stats.project(self.projection.as_ref())))
    }

    /// Tries to push `projection` down through `nested_loop_join`. If possible, performs the
    /// pushdown and returns a new [`NestedLoopJoinExec`] as the top plan which has projections
    /// as its children. Otherwise, returns `None`.
    fn try_swapping_with_projection(
        &self,
        projection: &ProjectionExec,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        // TODO: currently if there is projection in NestedLoopJoinExec, we can't push down projection to left or right input. Maybe we can pushdown the mixed projection later.
        if self.contains_projection() {
            return Ok(None);
        }

        let schema = self.schema();
        if let Some(JoinData {
            projected_left_child,
            projected_right_child,
            join_filter,
            ..
        }) = try_pushdown_through_join(
            projection,
            self.left(),
            self.right(),
            &[],
            &schema,
            self.filter(),
        )? {
            Ok(Some(Arc::new(NestedLoopJoinExec::try_new(
                Arc::new(projected_left_child),
                Arc::new(projected_right_child),
                join_filter,
                self.join_type(),
                // Returned early if projection is not None
                None,
            )?)))
        } else {
            try_embed_projection(projection, self)
        }
    }
}

impl EmbeddedProjection for NestedLoopJoinExec {
    fn with_projection(&self, projection: Option<Vec<usize>>) -> Result<Self> {
        self.with_projection(projection)
    }
}

/// Left (build-side) data
pub(crate) struct JoinLeftData {
    /// Build-side data collected to single batch
    batch: RecordBatch,
    /// Shared bitmap builder for visited left indices
    bitmap: SharedBitmapBuilder,
    /// Counter of running probe-threads, potentially able to update `bitmap`
    probe_threads_counter: AtomicUsize,
    /// Memory reservation for tracking batch and bitmap
    /// Cleared on `JoinLeftData` drop
    /// reservation is cleared on Drop
    #[expect(dead_code)]
    reservation: MemoryReservation,
}

impl JoinLeftData {
    pub(crate) fn new(
        batch: RecordBatch,
        bitmap: SharedBitmapBuilder,
        probe_threads_counter: AtomicUsize,
        reservation: MemoryReservation,
    ) -> Self {
        Self {
            batch,
            bitmap,
            probe_threads_counter,
            reservation,
        }
    }

    pub(crate) fn batch(&self) -> &RecordBatch {
        &self.batch
    }

    pub(crate) fn bitmap(&self) -> &SharedBitmapBuilder {
        &self.bitmap
    }

    /// Decrements counter of running threads, and returns `true`
    /// if caller is the last running thread
    pub(crate) fn report_probe_completed(&self) -> bool {
        self.probe_threads_counter.fetch_sub(1, Ordering::Relaxed) == 1
    }
}

/// Asynchronously collect input into a single batch, and creates `JoinLeftData` from it
async fn collect_left_input(
    stream: SendableRecordBatchStream,
    join_metrics: BuildProbeJoinMetrics,
    reservation: MemoryReservation,
    with_visited_left_side: bool,
    probe_threads_count: usize,
) -> Result<JoinLeftData> {
    let schema = stream.schema();

    // Load all batches and count the rows
    let (batches, metrics, reservation) = stream
        .try_fold(
            (Vec::new(), join_metrics, reservation),
            |(mut batches, metrics, reservation), batch| async {
                let batch_size = batch.get_array_memory_size();
                // Reserve memory for incoming batch
                reservation.try_grow(batch_size)?;
                // Update metrics
                metrics.build_mem_used.add(batch_size);
                metrics.build_input_batches.add(1);
                metrics.build_input_rows.add(batch.num_rows());
                // Push batch to output
                batches.push(batch);
                Ok((batches, metrics, reservation))
            },
        )
        .await?;

    let merged_batch = concat_batches(&schema, &batches)?;

    // Reserve memory for visited_left_side bitmap if required by join type
    let visited_left_side = if with_visited_left_side {
        let n_rows = merged_batch.num_rows();
        let buffer_size = n_rows.div_ceil(8);
        reservation.try_grow(buffer_size)?;
        metrics.build_mem_used.add(buffer_size);

        let mut buffer = BooleanBufferBuilder::new(n_rows);
        buffer.append_n(n_rows, false);
        buffer
    } else {
        BooleanBufferBuilder::new(0)
    };

    Ok(JoinLeftData::new(
        merged_batch,
        Mutex::new(visited_left_side),
        AtomicUsize::new(probe_threads_count),
        reservation,
    ))
}

/// States for join processing. See `poll_next()` comment for more details about
/// state transitions.
#[derive(Debug, Clone, Copy)]
enum NLJState {
    BufferingLeft,
    FetchingRight,
    ProbeRight,
    EmitRightUnmatched,
    EmitLeftUnmatched,
    /// Emit unmatched right rows using the global bitmap accumulated across
    /// all left chunks. Only used in memory-limited mode for join types that
    /// require tracking right-side matches in the final output (RIGHT, FULL,
    /// RIGHT SEMI, RIGHT ANTI, RIGHT MARK).
    EmitGlobalRightUnmatched,
    Done,
}
/// Shared data for the left-side spill fallback.
///
/// When the in-memory `OnceFut` path fails with OOM, the first partition
/// spills the entire left side to disk. This struct holds the spill file
/// reference so other partitions can read from the same file.
pub(crate) struct LeftSpillData {
    /// SpillManager used to read the spill file (has the left schema)
    spill_manager: SpillManager,
    /// The spill file containing all left-side batches
    spill_file: RefCountedTempFile,
    /// Left-side schema
    schema: SchemaRef,
}

/// Tracks the state of the memory-limited spill fallback for NLJ.
///
/// The NLJ always starts with the standard OnceFut path. If the in-memory
/// load fails with OOM and conditions allow, the operator falls back to a
/// multi-pass strategy where left data is loaded in chunks and the right
/// side is spilled to disk.
pub(crate) enum SpillState {
    /// Fallback is not possible (e.g., join type requires global right bitmap,
    /// or disk manager is disabled). OOM errors will propagate as-is.
    Disabled,

    /// Fallback is possible but not yet triggered. The operator is still
    /// attempting the standard OnceFut path. Holds the context needed to
    /// initiate fallback if OOM occurs.
    Pending {
        /// Left child plan for re-execution
        left_plan: Arc<dyn ExecutionPlan>,
        /// TaskContext for re-execution and SpillManager creation
        task_context: Arc<TaskContext>,
        /// Shared OnceAsync for left-side spill data. The first partition
        /// to initiate fallback spills the left side; others share the file.
        left_spill_data: Arc<OnceAsync<LeftSpillData>>,
    },

    /// Fallback has been triggered. Left data is being loaded in chunks
    /// and the right side is spilled to disk for re-scanning.
    Active(Box<SpillStateActive>),
}

/// State for active memory-limited spill execution.
/// Boxed inside [`SpillState::Active`] to reduce enum size.
pub(crate) struct SpillStateActive {
    /// Shared future for left-side spill data. All partitions wait on
    /// the same future — the first to poll triggers the actual spill.
    left_spill_fut: OnceFut<LeftSpillData>,
    /// Left input stream for incremental chunk reading (from spill file).
    /// None until `left_spill_fut` resolves.
    left_stream: Option<SendableRecordBatchStream>,
    /// Left-side schema (set once `left_spill_fut` resolves)
    left_schema: Option<SchemaRef>,
    /// Memory reservation for left-side buffering
    reservation: MemoryReservation,
    /// Accumulated left batches for the current chunk
    pending_batches: Vec<RecordBatch>,
    /// Right input that spills on the first pass and replays from spill later.
    right_input: ReplayableStreamSource,
    /// Per-batch accumulated right bitmaps across all left chunks.
    /// Index = right batch sequence number (0-based, non-empty batches only).
    /// Only populated when `should_track_unmatched_right` is true.
    global_right_bitmaps: Vec<BooleanBuffer>,
    /// Separate reservation for `global_right_bitmaps`. These buffers live
    /// for the full operator lifetime (not per-chunk), so they must be
    /// tracked separately from `reservation`, which gets `resize(0)`-ed
    /// between chunks.
    global_right_bitmaps_reservation: MemoryReservation,
    /// Current right batch sequence index within the current pass.
    right_batch_index: usize,
}

impl SpillStateActive {
    /// Merge a per-pass right bitmap into the global accumulator at the
    /// given batch index, growing the dedicated reservation when seeing
    /// a batch index for the first time.
    ///
    /// On first encounter of `idx`, the bitmap is stored as-is and its
    /// size is reserved. On subsequent encounters (later left chunk
    /// passes over the same right batch), the existing entry is OR-merged
    /// with `values`. Because `bitor` produces a buffer of the same bit
    /// length, the reservation does not need to be adjusted on merge.
    fn merge_current_right_bitmap(&mut self, idx: usize, values: BooleanBuffer) {
        if idx >= self.global_right_bitmaps.len() {
            // First encounter of this right batch — account memory and store.
            // The bitmap has one bit per right row, so for very large right
            // inputs the accumulated size can be non-negligible (e.g.,
            // 1M rows ≈ 125 KB per batch).
            // Use infallible `grow` because we must accept the bitmap to
            // preserve correctness — the fallback path has no other recourse.
            let bytes = values.len().div_ceil(8);
            self.global_right_bitmaps_reservation.grow(bytes);
            self.global_right_bitmaps.push(values);
        } else {
            // Subsequent left chunk pass — OR merge. Same bit length, so
            // no reservation adjustment is needed.
            self.global_right_bitmaps[idx] =
                self.global_right_bitmaps[idx].bitor(&values);
        }
    }
}

pub(crate) struct NestedLoopJoinStream {
    // ========================================================================
    // PROPERTIES:
    // Operator's properties that remain constant
    //
    // Note: The implementation uses the terms left/build-side table and
    // right/probe-side table interchangeably. Treating the left side as the
    // build side is a convention in DataFusion: the planner always tries to
    // swap the smaller table to the left side.
    // ========================================================================
    /// Output schema
    pub(crate) output_schema: Arc<Schema>,
    /// join filter
    pub(crate) join_filter: Option<JoinFilter>,
    /// type of the join
    pub(crate) join_type: JoinType,
    /// the probe-side(right) table data of the nested loop join
    /// `Option` is used because memory-limited path requires resetting it.
    pub(crate) right_data: Option<SendableRecordBatchStream>,
    /// the build-side table data of the nested loop join
    pub(crate) left_data: OnceFut<JoinLeftData>,
    /// Projection to construct the output schema from the left and right tables.
    /// Example:
    /// - output_schema: ['a', 'c']
    /// - left_schema: ['a', 'b']
    /// - right_schema: ['c']
    ///
    /// The column indices would be [(left, 0), (right, 0)] -- taking the left
    /// 0th column and right 0th column can construct the output schema.
    ///
    /// Note there are other columns ('b' in the example) still kept after
    /// projection pushdown; this is because they might be used to evaluate
    /// the join filter (e.g., `JOIN ON (b+c)>0`).
    pub(crate) column_indices: Vec<ColumnIndex>,
    /// Join execution metrics
    pub(crate) metrics: NestedLoopJoinMetrics,

    /// `batch_size` from configuration
    batch_size: usize,

    /// See comments in [`need_produce_right_in_final`] for more detail
    should_track_unmatched_right: bool,

    // ========================================================================
    // STATE FLAGS/BUFFERS:
    // Fields that hold intermediate data/flags during execution
    // ========================================================================
    /// State Tracking
    state: NLJState,
    /// Output buffer holds the join result to output. It will emit eagerly when
    /// the threshold is reached.
    output_buffer: Box<BatchCoalescer>,
    /// See comments in [`NLJState::Done`] for its purpose
    handled_empty_output: bool,

    // Buffer(left) side
    // -----------------
    /// The current buffered left data to join
    buffered_left_data: Option<Arc<JoinLeftData>>,
    /// Index into the left buffered batch. Used in `ProbeRight` state
    left_probe_idx: usize,
    /// Index into the left buffered batch. Used in `EmitLeftUnmatched` state
    left_emit_idx: usize,
    /// Should we go back to `BufferingLeft` state again after `EmitLeftUnmatched`
    /// state is over.
    left_exhausted: bool,
    /// If we can buffer all left data in one pass (false means memory-limited multi-pass)
    left_buffered_in_one_pass: bool,

    // Probe(right) side
    // -----------------
    /// The current probe batch to process
    current_right_batch: Option<RecordBatch>,
    // For right join, keep track of matched rows in `current_right_batch`
    // Constructed when fetching each new incoming right batch in `FetchingRight` state.
    current_right_batch_matched: Option<BooleanArray>,

    /// Memory-limited spill fallback state. See [`SpillState`] for details.
    spill_state: SpillState,
}

pub(crate) struct NestedLoopJoinMetrics {
    /// Join execution metrics
    pub(crate) join_metrics: BuildProbeJoinMetrics,
    /// Selectivity of the join: output_rows / (left_rows * right_rows)
    pub(crate) selectivity: RatioMetrics,
    /// Spill metrics for memory-limited execution
    pub(crate) spill_metrics: SpillMetrics,
}

impl NestedLoopJoinMetrics {
    pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self {
        Self {
            join_metrics: BuildProbeJoinMetrics::new(partition, metrics),
            selectivity: MetricBuilder::new(metrics)
                .with_type(MetricType::Summary)
                .ratio_metrics("selectivity", partition),
            spill_metrics: SpillMetrics::new(metrics, partition),
        }
    }
}

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

    /// See the comments [`NestedLoopJoinExec`] for high-level design ideas.
    ///
    /// # Implementation
    ///
    /// This function is the entry point of NLJ operator's state machine
    /// transitions. The rough state transition graph is as follow, for more
    /// details see the comment in each state's matching arm.
    ///
    /// ============================
    /// State transition graph:
    /// ============================
    ///
    /// (start) --> BufferingLeft
    /// ----------------------------
    /// BufferingLeft → FetchingRight
    ///
    /// FetchingRight → ProbeRight (if right batch available)
    /// FetchingRight → EmitLeftUnmatched (if right exhausted)
    ///
    /// ProbeRight → ProbeRight (next left row or after yielding output)
    /// ProbeRight → EmitRightUnmatched (for special join types like right join)
    /// ProbeRight → FetchingRight (done with the current right batch)
    ///
    /// EmitRightUnmatched → FetchingRight
    ///
    /// EmitLeftUnmatched → EmitLeftUnmatched (only process 1 chunk for each
    /// iteration)
    /// EmitLeftUnmatched → Done (if finished)
    /// ----------------------------
    /// Done → (end)
    fn poll_next(
        mut self: std::pin::Pin<&mut Self>,
        cx: &mut std::task::Context<'_>,
    ) -> Poll<Option<Self::Item>> {
        loop {
            match self.state {
                // # NLJState transitions
                // --> FetchingRight
                // This state will prepare the left side batches, next state
                // `FetchingRight` is responsible for preparing a single probe
                // side batch, before start joining.
                NLJState::BufferingLeft => {
                    debug!("[NLJState] Entering: {:?}", self.state);
                    // inside `collect_left_input` (the routine to buffer build
                    // -side batches), related metrics except build time will be
                    // updated.
                    // stop on drop
                    let build_metric = self.metrics.join_metrics.build_time.clone();
                    let _build_timer = build_metric.timer();

                    match self.handle_buffering_left(cx) {
                        ControlFlow::Continue(()) => continue,
                        ControlFlow::Break(poll) => return poll,
                    }
                }

                // # NLJState transitions:
                // 1. --> ProbeRight
                //    Start processing the join for the newly fetched right
                //    batch.
                // 2. --> EmitLeftUnmatched: When the right side input is exhausted, (maybe) emit
                //    unmatched left side rows.
                //
                // After fetching a new batch from the right side, it will
                // process all rows from the buffered left data:
                // ```text
                // for batch in right_side:
                //     for row in left_buffer:
                //         join(batch, row)
                // ```
                // Note: the implementation does this step incrementally,
                // instead of materializing all intermediate Cartesian products
                // at once in memory.
                //
                // So after the right side input is exhausted, the join phase
                // for the current buffered left data is finished. We can go to
                // the next `EmitLeftUnmatched` phase to check if there is any
                // special handling (e.g., in cases like left join).
                NLJState::FetchingRight => {
                    debug!("[NLJState] Entering: {:?}", self.state);
                    // stop on drop
                    let join_metric = self.metrics.join_metrics.join_time.clone();
                    let _join_timer = join_metric.timer();

                    match self.handle_fetching_right(cx) {
                        ControlFlow::Continue(()) => continue,
                        ControlFlow::Break(poll) => return poll,
                    }
                }

                // NLJState transitions:
                // 1. --> ProbeRight(1)
                //    If we have already buffered enough output to yield, it
                //    will first give back control to the parent state machine,
                //    then resume at the same place.
                // 2. --> ProbeRight(2)
                //    After probing one right batch, and evaluating the
                //    join filter on (left-row x right-batch), it will advance
                //    to the next left row, then re-enter the current state and
                //    continue joining.
                // 3. --> FetchRight
                //    After it has done with the current right batch (to join
                //    with all rows in the left buffer), it will go to
                //    FetchRight state to check what to do next.
                NLJState::ProbeRight => {
                    debug!("[NLJState] Entering: {:?}", self.state);

                    // stop on drop
                    let join_metric = self.metrics.join_metrics.join_time.clone();
                    let _join_timer = join_metric.timer();

                    match self.handle_probe_right() {
                        ControlFlow::Continue(()) => continue,
                        ControlFlow::Break(poll) => {
                            return self.metrics.join_metrics.baseline.record_poll(poll);
                        }
                    }
                }

                // In the `current_right_batch_matched` bitmap, all trues mean
                // it has been output by the join. In this state we have to
                // output unmatched rows for current right batch (with null
                // padding for left relation)
                // Precondition: we have checked the join type so that it's
                // possible to output right unmatched (e.g. it's right join)
                NLJState::EmitRightUnmatched => {
                    debug!("[NLJState] Entering: {:?}", self.state);

                    // stop on drop
                    let join_metric = self.metrics.join_metrics.join_time.clone();
                    let _join_timer = join_metric.timer();

                    match self.handle_emit_right_unmatched() {
                        ControlFlow::Continue(()) => continue,
                        ControlFlow::Break(poll) => {
                            return self.metrics.join_metrics.baseline.record_poll(poll);
                        }
                    }
                }

                // NLJState transitions:
                // 1. --> EmitLeftUnmatched(1)
                //    If we have already buffered enough output to yield, it
                //    will first give back control to the parent state machine,
                //    then resume at the same place.
                // 2. --> EmitLeftUnmatched(2)
                //    After processing some unmatched rows, it will re-enter
                //    the same state, to check if there are any more final
                //    results to output.
                // 3. --> Done
                //    It has processed all data, go to the final state and ready
                //    to exit.
                // 4. --> BufferingLeft (memory-limited mode only)
                //    When left data was loaded in chunks and more chunks remain,
                //    go back to BufferingLeft to load the next chunk.
                NLJState::EmitLeftUnmatched => {
                    debug!("[NLJState] Entering: {:?}", self.state);

                    // stop on drop
                    let join_metric = self.metrics.join_metrics.join_time.clone();
                    let _join_timer = join_metric.timer();

                    match self.handle_emit_left_unmatched() {
                        ControlFlow::Continue(()) => continue,
                        ControlFlow::Break(poll) => {
                            return self.metrics.join_metrics.baseline.record_poll(poll);
                        }
                    }
                }

                // Replay all right batches from spill and emit unmatched
                // right rows using the global bitmap accumulated across all
                // left chunks. Only entered in memory-limited mode for join
                // types where `should_track_unmatched_right` is true
                // (RIGHT, FULL, RIGHT SEMI, RIGHT ANTI, RIGHT MARK).
                NLJState::EmitGlobalRightUnmatched => {
                    debug!("[NLJState] Entering: {:?}", self.state);

                    let join_metric = self.metrics.join_metrics.join_time.clone();
                    let _join_timer = join_metric.timer();

                    match self.handle_emit_global_right_unmatched(cx) {
                        ControlFlow::Continue(()) => continue,
                        ControlFlow::Break(poll) => {
                            return self.metrics.join_metrics.baseline.record_poll(poll);
                        }
                    }
                }

                // The final state and the exit point
                NLJState::Done => {
                    debug!("[NLJState] Entering: {:?}", self.state);

                    // stop on drop
                    let join_metric = self.metrics.join_metrics.join_time.clone();
                    let _join_timer = join_metric.timer();
                    // counting it in join timer due to there might be some
                    // final resout batches to output in this state

                    let poll = self.handle_done();
                    return self.metrics.join_metrics.baseline.record_poll(poll);
                }
            }
        }
    }
}

impl RecordBatchStream for NestedLoopJoinStream {
    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.output_schema)
    }
}

impl NestedLoopJoinStream {
    #[expect(clippy::too_many_arguments)]
    pub(crate) fn new(
        schema: Arc<Schema>,
        filter: Option<JoinFilter>,
        join_type: JoinType,
        right_data: SendableRecordBatchStream,
        left_data: OnceFut<JoinLeftData>,
        column_indices: Vec<ColumnIndex>,
        metrics: NestedLoopJoinMetrics,
        batch_size: usize,
        spill_state: SpillState,
    ) -> Self {
        Self {
            output_schema: Arc::clone(&schema),
            join_filter: filter,
            join_type,
            right_data: Some(right_data),
            column_indices,
            left_data,
            metrics,
            buffered_left_data: None,
            output_buffer: Box::new(BatchCoalescer::new(schema, batch_size)),
            batch_size,
            current_right_batch: None,
            current_right_batch_matched: None,
            state: NLJState::BufferingLeft,
            left_probe_idx: 0,
            left_emit_idx: 0,
            left_exhausted: false,
            left_buffered_in_one_pass: true,
            handled_empty_output: false,
            should_track_unmatched_right: need_produce_right_in_final(join_type),
            spill_state,
        }
    }

    /// Returns true if this stream is operating in memory-limited mode
    fn is_memory_limited(&self) -> bool {
        matches!(self.spill_state, SpillState::Active(_))
    }

    /// Check if we can fall back to memory-limited mode on this error.
    fn can_fallback_to_spill(&self, error: &datafusion_common::DataFusionError) -> bool {
        matches!(self.spill_state, SpillState::Pending { .. })
            && matches!(
                error.find_root(),
                datafusion_common::DataFusionError::ResourcesExhausted(_)
            )
    }

    /// Switch from the standard OnceFut path to memory-limited mode.
    ///
    /// Uses the shared `left_spill_data` OnceAsync so that only the first
    /// partition to reach this point re-executes the left child and spills
    /// it to disk. Other partitions share the same spill file.
    fn initiate_fallback(&mut self) -> Result<()> {
        // Take ownership of Pending state
        let (left_plan, context, left_spill_data) =
            match std::mem::replace(&mut self.spill_state, SpillState::Disabled) {
                SpillState::Pending {
                    left_plan,
                    task_context,
                    left_spill_data,
                } => (left_plan, task_context, left_spill_data),
                _ => {
                    return internal_err!(
                        "initiate_fallback called in non-Pending spill state"
                    );
                }
            };

        // Use OnceAsync to ensure only the first partition spills the left
        // side. Other partitions will get the same OnceFut that resolves
        // to the shared spill file.
        let left_spill_fut = left_spill_data.try_once(|| {
            let plan = Arc::clone(&left_plan);
            let ctx = Arc::clone(&context);
            let spill_metrics = self.metrics.spill_metrics.clone();
            Ok(async move {
                let mut stream = plan.execute(0, Arc::clone(&ctx))?;
                let schema = stream.schema();
                let left_spill_manager = SpillManager::new(
                    ctx.runtime_env(),
                    spill_metrics,
                    Arc::clone(&schema),
                )
                .with_compression_type(ctx.session_config().spill_compression());

                let result = left_spill_manager
                    .spill_record_batch_stream_and_return_max_batch_memory(
                        &mut stream,
                        "NestedLoopJoin left spill",
                    )
                    .await?;

                match result {
                    Some((file, _max_batch_memory)) => Ok(LeftSpillData {
                        spill_manager: left_spill_manager,
                        spill_file: file,
                        schema,
                    }),
                    None => {
                        internal_err!("Left side produced no data to spill")
                    }
                }
            })
        })?;

        // Create reservation with can_spill for fair memory allocation
        let reservation = MemoryConsumer::new("NestedLoopJoinLoad[fallback]".to_string())
            .with_can_spill(true)
            .register(context.memory_pool());

        // Separate reservation for the global right bitmaps. These buffers
        // persist across all left chunks, whereas `reservation` is reset
        // between chunks via `resize(0)`.
        let global_right_bitmaps_reservation =
            MemoryConsumer::new("NestedLoopJoinGlobalRightBitmaps".to_string())
                .register(context.memory_pool());

        // Create SpillManager for right-side spilling
        let right_schema = self
            .right_data
            .as_ref()
            .expect("right_data must be present before fallback")
            .schema();
        let right_data = self
            .right_data
            .take()
            .expect("right_data must be present before fallback");
        let right_spill_manager = SpillManager::new(
            context.runtime_env(),
            self.metrics.spill_metrics.clone(),
            right_schema,
        )
        .with_compression_type(context.session_config().spill_compression());

        self.spill_state = SpillState::Active(Box::new(SpillStateActive {
            left_spill_fut,
            left_stream: None,
            left_schema: None,
            reservation,
            pending_batches: Vec::new(),
            right_input: ReplayableStreamSource::new(
                right_data,
                right_spill_manager,
                "NestedLoopJoin right spill",
            ),
            global_right_bitmaps: Vec::new(),
            global_right_bitmaps_reservation,
            right_batch_index: 0,
        }));

        // State stays BufferingLeft — next poll will enter
        // handle_buffering_left_memory_limited via is_memory_limited() check
        self.state = NLJState::BufferingLeft;

        Ok(())
    }

    // ==== State handler functions ====

    /// Handle BufferingLeft state - prepare left side batches.
    ///
    /// In standard mode, uses OnceFut to load all left data at once.
    /// In memory-limited mode, incrementally buffers left batches until the
    /// memory budget is reached or the left stream is exhausted.
    fn handle_buffering_left(
        &mut self,
        cx: &mut std::task::Context<'_>,
    ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>> {
        if self.is_memory_limited() {
            self.handle_buffering_left_memory_limited(cx)
        } else {
            // Standard path: use OnceFut
            match self.left_data.get_shared(cx) {
                Poll::Ready(Ok(left_data)) => {
                    self.buffered_left_data = Some(left_data);
                    self.left_exhausted = true;
                    self.state = NLJState::FetchingRight;
                    ControlFlow::Continue(())
                }
                Poll::Ready(Err(e)) => {
                    if self.can_fallback_to_spill(&e) {
                        debug!(
                            "NestedLoopJoin: OnceFut failed with OOM, \
                             falling back to memory-limited mode"
                        );
                        match self.initiate_fallback() {
                            Ok(()) => ControlFlow::Continue(()),
                            Err(fallback_err) => {
                                ControlFlow::Break(Poll::Ready(Some(Err(fallback_err))))
                            }
                        }
                    } else {
                        ControlFlow::Break(Poll::Ready(Some(Err(e))))
                    }
                }
                Poll::Pending => ControlFlow::Break(Poll::Pending),
            }
        }
    }

    /// Memory-limited path for handle_buffering_left.
    ///
    /// Incrementally polls the left stream and accumulates batches until:
    /// - Memory reservation fails (chunk is full, more data remains)
    /// - Left stream is exhausted (this is the last/only chunk)
    fn handle_buffering_left_memory_limited(
        &mut self,
        cx: &mut std::task::Context<'_>,
    ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>> {
        let SpillState::Active(active) = &mut self.spill_state else {
            unreachable!(
                "handle_buffering_left_memory_limited called without Active spill state"
            );
        };

        // On first entry (or after re-entry for a new chunk pass when
        // left_stream was consumed), wait for the shared left spill
        // future to resolve and then open a stream from the spill file.
        if active.left_stream.is_none() {
            match active.left_spill_fut.get_shared(cx) {
                Poll::Ready(Ok(spill_data)) => {
                    match spill_data
                        .spill_manager
                        .read_spill_as_stream(spill_data.spill_file.clone(), None)
                    {
                        Ok(stream) => {
                            active.left_schema = Some(Arc::clone(&spill_data.schema));
                            active.left_stream = Some(stream);
                        }
                        Err(e) => {
                            return ControlFlow::Break(Poll::Ready(Some(Err(e))));
                        }
                    }
                }
                Poll::Ready(Err(e)) => {
                    return ControlFlow::Break(Poll::Ready(Some(Err(e))));
                }
                Poll::Pending => {
                    return ControlFlow::Break(Poll::Pending);
                }
            }
        }

        let left_stream = active
            .left_stream
            .as_mut()
            .expect("left_stream must be set after spill future resolves");

        // Poll left stream for more batches.
        // Note: pending_batches may already contain a batch from the
        // previous chunk iteration (the batch that triggered the memory limit).
        loop {
            match left_stream.poll_next_unpin(cx) {
                Poll::Ready(Some(Ok(batch))) => {
                    if batch.num_rows() == 0 {
                        continue;
                    }
                    let batch_rows = batch.num_rows();
                    let batch_size = batch.get_array_memory_size();
                    let can_grow = active.reservation.try_grow(batch_size).is_ok();

                    if !can_grow && !active.pending_batches.is_empty() {
                        // Memory limit reached and we already have data.
                        // Push this batch into pending (it's already in memory)
                        // and stop buffering for this chunk.
                        active.pending_batches.push(batch);
                        self.left_exhausted = false;
                        self.left_buffered_in_one_pass = false;
                        break;
                    } else if !can_grow {
                        // No pending batches yet — we must accept this batch
                        // to make progress, even if it exceeds the budget.
                        active.reservation.grow(batch_size);
                    }

                    self.metrics.join_metrics.build_mem_used.add(batch_size);
                    self.metrics.join_metrics.build_input_batches.add(1);
                    self.metrics.join_metrics.build_input_rows.add(batch_rows);
                    active.pending_batches.push(batch);
                }
                Poll::Ready(Some(Err(e))) => {
                    return ControlFlow::Break(Poll::Ready(Some(Err(e))));
                }
                Poll::Ready(None) => {
                    // Left stream exhausted
                    self.left_exhausted = true;
                    break;
                }
                Poll::Pending => {
                    return ControlFlow::Break(Poll::Pending);
                }
            }
        }

        // If the left stream is fully exhausted, release its resources so the
        // upstream pipeline can be torn down before we move on to probing.
        if self.left_exhausted {
            active.left_stream = None;
        }

        if active.pending_batches.is_empty() {
            // No data at all — go directly to Done
            self.left_exhausted = true;
            self.state = NLJState::Done;
            return ControlFlow::Continue(());
        }

        let merged_batch = match concat_batches(
            active
                .left_schema
                .as_ref()
                .expect("left_schema must be set"),
            &active.pending_batches,
        ) {
            Ok(batch) => batch,
            Err(e) => {
                return ControlFlow::Break(Poll::Ready(Some(Err(e.into()))));
            }
        };
        active.pending_batches.clear();

        // Build visited bitmap if needed for this join type
        let with_visited = need_produce_result_in_final(self.join_type);
        let n_rows = merged_batch.num_rows();
        let visited_left_side = if with_visited {
            let buffer_size = n_rows.div_ceil(8);
            // Use infallible grow for bitmap — it's small
            active.reservation.grow(buffer_size);
            self.metrics.join_metrics.build_mem_used.add(buffer_size);
            let mut buffer = BooleanBufferBuilder::new(n_rows);
            buffer.append_n(n_rows, false);
            buffer
        } else {
            BooleanBufferBuilder::new(0)
        };

        // Create an empty reservation for JoinLeftData's RAII field.
        // The actual memory tracking is managed by the Active state's reservation.
        let dummy_reservation = active.reservation.new_empty();

        let left_data = JoinLeftData::new(
            merged_batch,
            Mutex::new(visited_left_side),
            // In memory-limited mode, only 1 probe thread per chunk
            AtomicUsize::new(1),
            dummy_reservation,
        );

        self.buffered_left_data = Some(Arc::new(left_data));

        active.right_batch_index = 0;
        match active.right_input.open_pass() {
            Ok(stream) => {
                self.right_data = Some(stream);
            }
            Err(e) => {
                return ControlFlow::Break(Poll::Ready(Some(Err(e))));
            }
        }

        self.state = NLJState::FetchingRight;
        ControlFlow::Continue(())
    }

    /// Handle FetchingRight state - fetch next right batch and prepare for processing.
    ///
    /// In memory-limited mode during the first pass, each right batch is also
    /// written to a spill file so it can be re-read on subsequent passes.
    fn handle_fetching_right(
        &mut self,
        cx: &mut std::task::Context<'_>,
    ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>> {
        match self
            .right_data
            .as_mut()
            .expect("right_data must be present while fetching right")
            .poll_next_unpin(cx)
        {
            Poll::Ready(result) => match result {
                Some(Ok(right_batch)) => {
                    // Update metrics
                    let right_batch_rows = right_batch.num_rows();
                    self.metrics.join_metrics.input_rows.add(right_batch_rows);
                    self.metrics.join_metrics.input_batches.add(1);

                    // Skip the empty batch
                    if right_batch_rows == 0 {
                        return ControlFlow::Continue(());
                    }

                    self.current_right_batch = Some(right_batch);

                    // Prepare right bitmap
                    if self.should_track_unmatched_right {
                        let zeroed_buf = BooleanBuffer::new_unset(right_batch_rows);
                        self.current_right_batch_matched =
                            Some(BooleanArray::new(zeroed_buf, None));
                    }

                    self.left_probe_idx = 0;
                    self.state = NLJState::ProbeRight;
                    ControlFlow::Continue(())
                }
                Some(Err(e)) => ControlFlow::Break(Poll::Ready(Some(Err(e)))),
                None => {
                    self.state = NLJState::EmitLeftUnmatched;
                    ControlFlow::Continue(())
                }
            },
            Poll::Pending => ControlFlow::Break(Poll::Pending),
        }
    }

    /// Handle ProbeRight state - process current probe batch
    fn handle_probe_right(&mut self) -> ControlFlow<Poll<Option<Result<RecordBatch>>>> {
        // Return any completed batches first
        if let Some(poll) = self.maybe_flush_ready_batch() {
            return ControlFlow::Break(poll);
        }

        // Process current probe state
        match self.process_probe_batch() {
            // State unchanged (ProbeRight)
            // Continue probing until we have done joining the
            // current right batch with all buffered left rows.
            Ok(true) => ControlFlow::Continue(()),
            // To next FetchRightState
            // We have finished joining
            // (cur_right_batch x buffered_left_batches)
            Ok(false) => {
                // Left exhausted, transition to FetchingRight
                self.left_probe_idx = 0;

                // Selectivity Metric: Update total possibilities for the batch (left_rows * right_rows)
                // If memory-limited execution is implemented, this logic must be updated accordingly.
                if let (Ok(left_data), Some(right_batch)) =
                    (self.get_left_data(), self.current_right_batch.as_ref())
                {
                    let left_rows = left_data.batch().num_rows();
                    let right_rows = right_batch.num_rows();
                    self.metrics.selectivity.add_total(left_rows * right_rows);
                }

                if self.should_track_unmatched_right {
                    debug_assert!(
                        self.current_right_batch_matched.is_some(),
                        "If it's required to track matched rows in the right input, the right bitmap must be present"
                    );
                    self.state = NLJState::EmitRightUnmatched;
                } else {
                    self.current_right_batch = None;
                    self.state = NLJState::FetchingRight;
                }
                ControlFlow::Continue(())
            }
            Err(e) => ControlFlow::Break(Poll::Ready(Some(Err(e)))),
        }
    }

    /// Handle EmitRightUnmatched state - emit unmatched right rows.
    ///
    /// In memory-limited mode, instead of emitting unmatched right rows
    /// per-batch (which would be incorrect since more left chunks may
    /// match those rows), we merge the bitmap into the global accumulator
    /// and defer emission to `EmitGlobalRightUnmatched`.
    fn handle_emit_right_unmatched(
        &mut self,
    ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>> {
        // In memory-limited mode, merge bitmap into global and move on
        if self.is_memory_limited() {
            debug_assert!(
                self.current_right_batch_matched.is_some(),
                "right bitmap must be present"
            );
            let bitmap = std::mem::take(&mut self.current_right_batch_matched)
                .expect("right bitmap should be available");
            let (values, _nulls) = bitmap.into_parts();

            if let SpillState::Active(ref mut active) = self.spill_state {
                let idx = active.right_batch_index;
                active.merge_current_right_bitmap(idx, values);
                active.right_batch_index += 1;
            }

            self.current_right_batch = None;
            self.state = NLJState::FetchingRight;
            return ControlFlow::Continue(());
        }

        // Standard (single-pass) mode: emit unmatched right rows immediately
        // Return any completed batches first
        if let Some(poll) = self.maybe_flush_ready_batch() {
            return ControlFlow::Break(poll);
        }

        debug_assert!(
            self.current_right_batch_matched.is_some()
                && self.current_right_batch.is_some(),
            "This state is yielding output for unmatched rows in the current right batch, so both the right batch and the bitmap must be present"
        );
        match self.process_right_unmatched() {
            Ok(Some(batch)) => match self.output_buffer.push_batch(batch) {
                Ok(()) => {
                    debug_assert!(self.current_right_batch.is_none());
                    self.state = NLJState::FetchingRight;
                    ControlFlow::Continue(())
                }
                Err(e) => ControlFlow::Break(Poll::Ready(Some(arrow_err!(e)))),
            },
            Ok(None) => {
                debug_assert!(self.current_right_batch.is_none());
                self.state = NLJState::FetchingRight;
                ControlFlow::Continue(())
            }
            Err(e) => ControlFlow::Break(Poll::Ready(Some(Err(e)))),
        }
    }

    /// Handle EmitLeftUnmatched state - emit unmatched left rows.
    ///
    /// In memory-limited mode, after processing all unmatched rows for the
    /// current left chunk, transitions back to `BufferingLeft` to load the
    /// next chunk (if the left stream is not yet exhausted).
    fn handle_emit_left_unmatched(
        &mut self,
    ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>> {
        // Return any completed batches first
        if let Some(poll) = self.maybe_flush_ready_batch() {
            return ControlFlow::Break(poll);
        }

        // Process current unmatched state
        match self.process_left_unmatched() {
            // State unchanged (EmitLeftUnmatched)
            // Continue processing until we have processed all unmatched rows
            Ok(true) => ControlFlow::Continue(()),
            // We have finished processing all unmatched rows for this chunk
            Ok(false) => match self.output_buffer.finish_buffered_batch() {
                Ok(()) => {
                    // Flush any completed batch before transitioning.
                    // This is critical for the memory-limited path: the
                    // ProbeRight results must be emitted before we discard
                    // the current chunk and load the next one.
                    if let Some(poll) = self.maybe_flush_ready_batch() {
                        return ControlFlow::Break(poll);
                    }

                    if !self.left_exhausted && self.is_memory_limited() {
                        // More left data to process — free current chunk and
                        // go back to BufferingLeft for the next chunk
                        if let SpillState::Active(ref active) = self.spill_state {
                            active.reservation.resize(0);
                        }
                        self.buffered_left_data = None;
                        self.left_probe_idx = 0;
                        self.left_emit_idx = 0;
                        self.state = NLJState::BufferingLeft;
                    } else if self.is_memory_limited()
                        && self.should_track_unmatched_right
                    {
                        // All left chunks done — emit global right unmatched.
                        // Drop the exhausted right stream so that
                        // EmitGlobalRightUnmatched opens a fresh replay pass
                        // from the spill file. (process_left_unmatched_range
                        // already ran with right_data still set, so its
                        // schema access is not affected.)
                        self.right_data = None;
                        self.state = NLJState::EmitGlobalRightUnmatched;
                    } else {
                        self.state = NLJState::Done;
                    }
                    ControlFlow::Continue(())
                }
                Err(e) => ControlFlow::Break(Poll::Ready(Some(arrow_err!(e)))),
            },
            Err(e) => ControlFlow::Break(Poll::Ready(Some(Err(e)))),
        }
    }

    /// Handle EmitGlobalRightUnmatched state.
    ///
    /// Replays all right batches from the spill file and emits unmatched
    /// right rows using the global bitmap accumulated across all left chunks.
    fn handle_emit_global_right_unmatched(
        &mut self,
        cx: &mut std::task::Context<'_>,
    ) -> ControlFlow<Poll<Option<Result<RecordBatch>>>> {
        // Flush any completed batches first
        if let Some(poll) = self.maybe_flush_ready_batch() {
            return ControlFlow::Break(poll);
        }

        // On first entry, open a new replay pass on the right input
        if self.right_data.is_none() {
            let SpillState::Active(ref mut active) = self.spill_state else {
                unreachable!("EmitGlobalRightUnmatched without Active spill state");
            };
            active.right_batch_index = 0;
            match active.right_input.open_pass() {
                Ok(stream) => {
                    self.right_data = Some(stream);
                }
                Err(e) => {
                    return ControlFlow::Break(Poll::Ready(Some(Err(e))));
                }
            }
        }

        // Poll the replay stream for the next right batch
        match self
            .right_data
            .as_mut()
            .expect("right_data must be present")
            .poll_next_unpin(cx)
        {
            Poll::Ready(Some(Ok(right_batch))) => {
                if right_batch.num_rows() == 0 {
                    return ControlFlow::Continue(());
                }

                let SpillState::Active(ref mut active) = self.spill_state else {
                    unreachable!();
                };
                let idx = active.right_batch_index;
                active.right_batch_index += 1;

                // Build BooleanArray from the global bitmap
                let bitmap = if idx < active.global_right_bitmaps.len() {
                    BooleanArray::new(active.global_right_bitmaps[idx].clone(), None)
                } else {
                    // Batch never seen — treat all rows as unmatched
                    BooleanArray::new(
                        BooleanBuffer::new_unset(right_batch.num_rows()),
                        None,
                    )
                };

                let left_schema = Arc::clone(
                    active
                        .left_schema
                        .as_ref()
                        .expect("left_schema must be set"),
                );

                match build_unmatched_batch(
                    &self.output_schema,
                    &right_batch,
                    bitmap,
                    &left_schema,
                    &self.column_indices,
                    self.join_type,
                    JoinSide::Right,
                ) {
                    Ok(Some(batch)) => match self.output_buffer.push_batch(batch) {
                        Ok(()) => ControlFlow::Continue(()),
                        Err(e) => ControlFlow::Break(Poll::Ready(Some(arrow_err!(e)))),
                    },
                    Ok(None) => ControlFlow::Continue(()),
                    Err(e) => ControlFlow::Break(Poll::Ready(Some(Err(e)))),
                }
            }
            Poll::Ready(Some(Err(e))) => ControlFlow::Break(Poll::Ready(Some(Err(e)))),
            Poll::Ready(None) => {
                // All right batches replayed
                match self.output_buffer.finish_buffered_batch() {
                    Ok(()) => {
                        self.state = NLJState::Done;
                        ControlFlow::Continue(())
                    }
                    Err(e) => ControlFlow::Break(Poll::Ready(Some(arrow_err!(e)))),
                }
            }
            Poll::Pending => ControlFlow::Break(Poll::Pending),
        }
    }

    /// Handle Done state - final state processing
    fn handle_done(&mut self) -> Poll<Option<Result<RecordBatch>>> {
        // Return any remaining completed batches before final termination
        if let Some(poll) = self.maybe_flush_ready_batch() {
            return poll;
        }

        // HACK for the doc test in https://github.com/apache/datafusion/blob/main/datafusion/core/src/dataframe/mod.rs#L1265
        // If this operator directly return `Poll::Ready(None)`
        // for empty result, the final result will become an empty
        // batch with empty schema, however the expected result
        // should be with the expected schema for this operator
        if !self.handled_empty_output {
            let zero_count = Count::new();
            if *self.metrics.join_metrics.baseline.output_rows() == zero_count {
                let empty_batch = RecordBatch::new_empty(Arc::clone(&self.output_schema));
                self.handled_empty_output = true;
                return Poll::Ready(Some(Ok(empty_batch)));
            }
        }

        Poll::Ready(None)
    }

    // ==== Core logic handling for each state ====

    /// Returns bool to indicate should it continue probing
    /// true -> continue in the same ProbeRight state
    /// false -> It has done with the (buffered_left x cur_right_batch), go to
    /// next state (ProbeRight)
    fn process_probe_batch(&mut self) -> Result<bool> {
        let left_data = Arc::clone(self.get_left_data()?);
        let right_batch = self
            .current_right_batch
            .as_ref()
            .ok_or_else(|| internal_datafusion_err!("Right batch should be available"))?
            .clone();

        // stop probing, the caller will go to the next state
        if self.left_probe_idx >= left_data.batch().num_rows() {
            return Ok(false);
        }

        // ========
        // Join (l_row x right_batch)
        // and push the result into output_buffer
        // ========

        // Special case:
        // When the right batch is very small, join with multiple left rows at once,
        //
        // The regular implementation is not efficient if the plan's right child is
        // very small (e.g. 1 row total), because inside the inner loop of NLJ, it's
        // handling one input right batch at once, if it's not large enough, the
        // overheads like filter evaluation can't be amortized through vectorization.
        debug_assert_ne!(
            right_batch.num_rows(),
            0,
            "When fetching the right batch, empty batches will be skipped"
        );

        let l_row_cnt_ratio = self.batch_size / right_batch.num_rows();
        if l_row_cnt_ratio > 10 {
            // Calculate max left rows to handle at once. This operator tries to handle
            // up to `datafusion.execution.batch_size` rows at once in the intermediate
            // batch.
            let l_row_count = std::cmp::min(
                l_row_cnt_ratio,
                left_data.batch().num_rows() - self.left_probe_idx,
            );

            debug_assert!(
                l_row_count != 0,
                "This function should only be entered when there are remaining left rows to process"
            );
            let joined_batch = self.process_left_range_join(
                &left_data,
                &right_batch,
                self.left_probe_idx,
                l_row_count,
            )?;

            if let Some(batch) = joined_batch {
                self.output_buffer.push_batch(batch)?;
            }

            self.left_probe_idx += l_row_count;

            return Ok(true);
        }

        let l_idx = self.left_probe_idx;
        let joined_batch =
            self.process_single_left_row_join(&left_data, &right_batch, l_idx)?;

        if let Some(batch) = joined_batch {
            self.output_buffer.push_batch(batch)?;
        }

        // ==== Prepare for the next iteration ====

        // Advance left cursor
        self.left_probe_idx += 1;

        // Return true to continue probing
        Ok(true)
    }

    /// Process [l_start_index, l_start_index + l_count) JOIN right_batch
    /// Returns a RecordBatch containing the join results (None if empty)
    ///
    /// Side Effect: If the join type requires, left or right side matched bitmap
    /// will be set for matched indices.
    fn process_left_range_join(
        &mut self,
        left_data: &JoinLeftData,
        right_batch: &RecordBatch,
        l_start_index: usize,
        l_row_count: usize,
    ) -> Result<Option<RecordBatch>> {
        // Construct the Cartesian product between the specified range of left rows
        // and the entire right_batch. First, it calculates the index vectors, then
        // materializes the intermediate batch, and finally applies the join filter
        // to it.
        // -----------------------------------------------------------
        let right_rows = right_batch.num_rows();
        let total_rows = l_row_count * right_rows;

        // Build index arrays for cartesian product: left_range X right_batch
        let left_indices: UInt32Array =
            UInt32Array::from_iter_values((0..l_row_count).flat_map(|i| {
                std::iter::repeat_n((l_start_index + i) as u32, right_rows)
            }));
        let right_indices: UInt32Array = UInt32Array::from_iter_values(
            (0..l_row_count).flat_map(|_| 0..right_rows as u32),
        );

        debug_assert!(
            left_indices.len() == right_indices.len()
                && right_indices.len() == total_rows,
            "The length or cartesian product should be (left_size * right_size)",
        );

        // Evaluate the join filter (if any) over an intermediate batch built
        // using the filter's own schema/column indices.
        let bitmap_combined = if let Some(filter) = &self.join_filter {
            // Build the intermediate batch for filter evaluation
            let intermediate_batch = if filter.schema.fields().is_empty() {
                // Constant predicate (e.g., TRUE/FALSE). Use an empty schema with row_count
                create_record_batch_with_empty_schema(
                    Arc::new((*filter.schema).clone()),
                    total_rows,
                )?
            } else {
                let mut filter_columns: Vec<Arc<dyn Array>> =
                    Vec::with_capacity(filter.column_indices().len());
                for column_index in filter.column_indices() {
                    let array = if column_index.side == JoinSide::Left {
                        let col = left_data.batch().column(column_index.index);
                        take(col.as_ref(), &left_indices, None)?
                    } else {
                        let col = right_batch.column(column_index.index);
                        take(col.as_ref(), &right_indices, None)?
                    };
                    filter_columns.push(array);
                }

                RecordBatch::try_new(Arc::new((*filter.schema).clone()), filter_columns)?
            };

            let filter_result = filter
                .expression()
                .evaluate(&intermediate_batch)?
                .into_array(intermediate_batch.num_rows())?;
            let filter_arr = as_boolean_array(&filter_result)?;

            // Combine with null bitmap to get a unified mask
            boolean_mask_from_filter(filter_arr)
        } else {
            // No filter: all pairs match
            BooleanArray::from(vec![true; total_rows])
        };

        // Update the global left or right bitmap for matched indices
        // -----------------------------------------------------------

        // None means we don't have to update left bitmap for this join type
        let mut left_bitmap = if need_produce_result_in_final(self.join_type) {
            Some(left_data.bitmap().lock())
        } else {
            None
        };

        // 'local' meaning: we want to collect 'is_matched' flag for the current
        // right batch, after it has joining all of the left buffer, here it's only
        // the partial result for joining given left range
        let mut local_right_bitmap = if self.should_track_unmatched_right {
            let mut current_right_batch_bitmap = BooleanBufferBuilder::new(right_rows);
            // Ensure builder has logical length so set_bit is in-bounds
            current_right_batch_bitmap.append_n(right_rows, false);
            Some(current_right_batch_bitmap)
        } else {
            None
        };

        // Set the matched bit for left and right side bitmap
        for (i, is_matched) in bitmap_combined.iter().enumerate() {
            let is_matched = is_matched.ok_or_else(|| {
                internal_datafusion_err!("Must be Some after the previous combining step")
            })?;

            let l_index = l_start_index + i / right_rows;
            let r_index = i % right_rows;

            if let Some(bitmap) = left_bitmap.as_mut()
                && is_matched
            {
                // Map local index back to absolute left index within the batch
                bitmap.set_bit(l_index, true);
            }

            if let Some(bitmap) = local_right_bitmap.as_mut()
                && is_matched
            {
                bitmap.set_bit(r_index, true);
            }
        }

        // Apply the local right bitmap to the global bitmap
        if self.should_track_unmatched_right {
            // Remember to put it back after update
            let global_right_bitmap =
                std::mem::take(&mut self.current_right_batch_matched).ok_or_else(
                    || internal_datafusion_err!("right batch's bitmap should be present"),
                )?;
            let (buf, nulls) = global_right_bitmap.into_parts();
            debug_assert!(nulls.is_none());

            let current_right_bitmap = local_right_bitmap
                .ok_or_else(|| {
                    internal_datafusion_err!(
                        "Should be Some if the current join type requires right bitmap"
                    )
                })?
                .finish();
            let updated_global_right_bitmap = buf.bitor(&current_right_bitmap);

            self.current_right_batch_matched =
                Some(BooleanArray::new(updated_global_right_bitmap, None));
        }

        // For the following join types: only bitmaps are updated; do not emit rows now
        if matches!(
            self.join_type,
            JoinType::LeftAnti
                | JoinType::LeftSemi
                | JoinType::LeftMark
                | JoinType::RightAnti
                | JoinType::RightMark
                | JoinType::RightSemi
        ) {
            return Ok(None);
        }

        // Build the projected output batch (using output schema/column_indices),
        // then apply the bitmap filter to it.
        if self.output_schema.fields().is_empty() {
            // Empty projection: only row count matters
            let row_count = bitmap_combined.true_count();
            return Ok(Some(create_record_batch_with_empty_schema(
                Arc::clone(&self.output_schema),
                row_count,
            )?));
        }

        let mut out_columns: Vec<Arc<dyn Array>> =
            Vec::with_capacity(self.output_schema.fields().len());
        for column_index in &self.column_indices {
            let array = if column_index.side == JoinSide::Left {
                let col = left_data.batch().column(column_index.index);
                take(col.as_ref(), &left_indices, None)?
            } else {
                let col = right_batch.column(column_index.index);
                take(col.as_ref(), &right_indices, None)?
            };
            out_columns.push(array);
        }
        let pre_filtered =
            RecordBatch::try_new(Arc::clone(&self.output_schema), out_columns)?;
        let filtered = filter_record_batch(&pre_filtered, &bitmap_combined)?;
        Ok(Some(filtered))
    }

    /// Process a single left row join with the current right batch.
    /// Returns a RecordBatch containing the join results (None if empty)
    ///
    /// Side Effect: If the join type requires, left or right side matched bitmap
    /// will be set for matched indices.
    fn process_single_left_row_join(
        &mut self,
        left_data: &JoinLeftData,
        right_batch: &RecordBatch,
        l_index: usize,
    ) -> Result<Option<RecordBatch>> {
        let right_row_count = right_batch.num_rows();
        if right_row_count == 0 {
            return Ok(None);
        }

        let cur_right_bitmap = if let Some(filter) = &self.join_filter {
            apply_filter_to_row_join_batch(
                left_data.batch(),
                l_index,
                right_batch,
                filter,
            )?
        } else {
            BooleanArray::from(vec![true; right_row_count])
        };

        self.update_matched_bitmap(l_index, &cur_right_bitmap)?;

        // For the following join types: here we only have to set the left/right
        // bitmap, and no need to output result
        if matches!(
            self.join_type,
            JoinType::LeftAnti
                | JoinType::LeftSemi
                | JoinType::LeftMark
                | JoinType::RightAnti
                | JoinType::RightMark
                | JoinType::RightSemi
        ) {
            return Ok(None);
        }

        if !cur_right_bitmap.has_true() {
            // If none of the pairs has passed the join predicate/filter
            Ok(None)
        } else {
            // Use the optimized approach similar to build_intermediate_batch_for_single_left_row
            let join_batch = build_row_join_batch(
                &self.output_schema,
                left_data.batch(),
                l_index,
                right_batch,
                Some(cur_right_bitmap),
                &self.column_indices,
                JoinSide::Left,
            )?;
            Ok(join_batch)
        }
    }

    /// Returns bool to indicate should it continue processing unmatched rows
    /// true -> continue in the same EmitLeftUnmatched state
    /// false -> next state (Done)
    fn process_left_unmatched(&mut self) -> Result<bool> {
        let left_data = self.get_left_data()?;
        let left_batch = left_data.batch();

        // ========
        // Check early return conditions
        // ========

        // Early return if join type can't have unmatched rows
        let join_type_no_produce_left = !need_produce_result_in_final(self.join_type);
        // Early return if another thread is already processing unmatched rows
        let handled_by_other_partition =
            self.left_emit_idx == 0 && !left_data.report_probe_completed();
        // Stop processing unmatched rows, the caller will go to the next state
        let finished = self.left_emit_idx >= left_batch.num_rows();

        if join_type_no_produce_left || handled_by_other_partition || finished {
            return Ok(false);
        }

        // ========
        // Process unmatched rows and push the result into output_buffer
        // Each time, the number to process is up to batch size
        // ========
        let start_idx = self.left_emit_idx;
        let end_idx = std::cmp::min(start_idx + self.batch_size, left_batch.num_rows());

        if let Some(batch) =
            self.process_left_unmatched_range(left_data, start_idx, end_idx)?
        {
            self.output_buffer.push_batch(batch)?;
        }

        // ==== Prepare for the next iteration ====
        self.left_emit_idx = end_idx;

        // Return true to continue processing unmatched rows
        Ok(true)
    }

    /// Process unmatched rows from the left data within the specified range.
    /// Returns a RecordBatch containing the unmatched rows (None if empty).
    ///
    /// # Arguments
    /// * `left_data` - The left side data containing the batch and bitmap
    /// * `start_idx` - Start index (inclusive) of the range to process
    /// * `end_idx` - End index (exclusive) of the range to process
    ///
    /// # Safety
    /// The caller is responsible for ensuring that `start_idx` and `end_idx` are
    /// within valid bounds of the left batch. This function does not perform
    /// bounds checking.
    fn process_left_unmatched_range(
        &self,
        left_data: &JoinLeftData,
        start_idx: usize,
        end_idx: usize,
    ) -> Result<Option<RecordBatch>> {
        if start_idx == end_idx {
            return Ok(None);
        }

        // Slice both left batch, and bitmap to range [start_idx, end_idx)
        // The range is bit index (not byte)
        let left_batch = left_data.batch();
        let left_batch_sliced = left_batch.slice(start_idx, end_idx - start_idx);

        // Can this be more efficient?
        let mut bitmap_sliced = BooleanBufferBuilder::new(end_idx - start_idx);
        bitmap_sliced.append_n(end_idx - start_idx, false);
        let bitmap = left_data.bitmap().lock();
        for i in start_idx..end_idx {
            assert!(
                i - start_idx < bitmap_sliced.capacity(),
                "DBG: {start_idx}, {end_idx}"
            );
            bitmap_sliced.set_bit(i - start_idx, bitmap.get_bit(i));
        }
        let bitmap_sliced = BooleanArray::new(bitmap_sliced.finish(), None);

        let right_schema = self
            .right_data
            .as_ref()
            .expect("right_data must be present when building unmatched batch")
            .schema();
        build_unmatched_batch(
            &self.output_schema,
            &left_batch_sliced,
            bitmap_sliced,
            &right_schema,
            &self.column_indices,
            self.join_type,
            JoinSide::Left,
        )
    }

    /// Process unmatched rows from the current right batch and reset the bitmap.
    /// Returns a RecordBatch containing the unmatched right rows (None if empty).
    fn process_right_unmatched(&mut self) -> Result<Option<RecordBatch>> {
        // ==== Take current right batch and its bitmap ====
        let right_batch_bitmap: BooleanArray =
            std::mem::take(&mut self.current_right_batch_matched).ok_or_else(|| {
                internal_datafusion_err!("right bitmap should be available")
            })?;

        let right_batch = self.current_right_batch.take();
        let cur_right_batch = unwrap_or_internal_err!(right_batch);

        let left_data = self.get_left_data()?;
        let left_schema = left_data.batch().schema();

        let res = build_unmatched_batch(
            &self.output_schema,
            &cur_right_batch,
            right_batch_bitmap,
            &left_schema,
            &self.column_indices,
            self.join_type,
            JoinSide::Right,
        );

        // ==== Clean-up ====
        self.current_right_batch_matched = None;

        res
    }

    // ==== Utilities ====

    /// Get the build-side data of the left input, errors if it's None
    fn get_left_data(&self) -> Result<&Arc<JoinLeftData>> {
        self.buffered_left_data
            .as_ref()
            .ok_or_else(|| internal_datafusion_err!("LeftData should be available"))
    }

    /// Flush the `output_buffer` if there are batches ready to output
    /// None if no result batch ready.
    fn maybe_flush_ready_batch(&mut self) -> Option<Poll<Option<Result<RecordBatch>>>> {
        if self.output_buffer.has_completed_batch()
            && let Some(batch) = self.output_buffer.next_completed_batch()
        {
            // Update output rows for selectivity metric
            let output_rows = batch.num_rows();
            self.metrics.selectivity.add_part(output_rows);

            return Some(Poll::Ready(Some(Ok(batch))));
        }

        None
    }

    /// After joining (l_index@left_buffer x current_right_batch), it will result
    /// in a bitmap (the same length as current_right_batch) as the join match
    /// result. Use this bitmap to update the global bitmap, for special join
    /// types like full joins.
    ///
    /// Example:
    /// After joining l_index=1 (1-indexed row in the left buffer), and the
    /// current right batch with 3 elements, this function will be called with
    /// arguments: l_index = 1, r_matched = [false, false, true]
    /// - If the join type is FullJoin, the 1-index in the left bitmap will be
    ///   set to true, and also the right bitmap will be bitwise-ORed with the
    ///   input r_matched bitmap.
    /// - For join types that don't require output unmatched rows, this
    ///   function can be a no-op. For inner joins, this function is a no-op; for left
    ///   joins, only the left bitmap may be updated.
    fn update_matched_bitmap(
        &mut self,
        l_index: usize,
        r_matched_bitmap: &BooleanArray,
    ) -> Result<()> {
        let left_data = self.get_left_data()?;

        // 1. Maybe update the left bitmap
        if need_produce_result_in_final(self.join_type) && r_matched_bitmap.has_true() {
            let mut bitmap = left_data.bitmap().lock();
            bitmap.set_bit(l_index, true);
        }

        // 2. Maybe update the right bitmap
        if self.should_track_unmatched_right {
            debug_assert!(self.current_right_batch_matched.is_some());
            // after bit-wise or, it will be put back
            let right_bitmap = std::mem::take(&mut self.current_right_batch_matched)
                .ok_or_else(|| {
                    internal_datafusion_err!("right batch's bitmap should be present")
                })?;
            let (buf, nulls) = right_bitmap.into_parts();
            debug_assert!(nulls.is_none());
            let updated_right_bitmap = buf.bitor(r_matched_bitmap.values());

            self.current_right_batch_matched =
                Some(BooleanArray::new(updated_right_bitmap, None));
        }

        Ok(())
    }
}

// ==== Utilities ====

/// Apply the join filter between:
/// (l_index th row in left buffer) x (right batch)
/// Returns a bitmap, with successfully joined indices set to true
fn apply_filter_to_row_join_batch(
    left_batch: &RecordBatch,
    l_index: usize,
    right_batch: &RecordBatch,
    filter: &JoinFilter,
) -> Result<BooleanArray> {
    debug_assert!(left_batch.num_rows() != 0 && right_batch.num_rows() != 0);

    let intermediate_batch = if filter.schema.fields().is_empty() {
        // If filter is constant (e.g. literal `true`), empty batch can be used
        // in the later filter step.
        create_record_batch_with_empty_schema(
            Arc::new((*filter.schema).clone()),
            right_batch.num_rows(),
        )?
    } else {
        build_row_join_batch(
            &filter.schema,
            left_batch,
            l_index,
            right_batch,
            None,
            &filter.column_indices,
            JoinSide::Left,
        )?
        .ok_or_else(|| internal_datafusion_err!("This function assume input batch is not empty, so the intermediate batch can't be empty too"))?
    };

    let filter_result = filter
        .expression()
        .evaluate(&intermediate_batch)?
        .into_array(intermediate_batch.num_rows())?;
    let filter_arr = as_boolean_array(&filter_result)?;

    // Convert boolean array with potential nulls into a unified mask bitmap
    let bitmap_combined = boolean_mask_from_filter(filter_arr);

    Ok(bitmap_combined)
}

/// Convert a boolean filter array into a unified mask bitmap.
///
/// Caution: The filter result is NOT a bitmap; it contains true/false/null values.
/// For example, `1 < NULL` evaluates to NULL. Therefore, we must combine (AND)
/// the boolean array with its null bitmap to construct a unified bitmap.
#[inline]
fn boolean_mask_from_filter(filter_arr: &BooleanArray) -> BooleanArray {
    let (values, nulls) = filter_arr.clone().into_parts();
    match nulls {
        Some(nulls) => BooleanArray::new(nulls.inner() & &values, None),
        None => BooleanArray::new(values, None),
    }
}

/// This function performs the following steps:
/// 1. Apply filter to probe-side batch
/// 2. Broadcast the left row (build_side_batch\[build_side_index\]) to the
///    filtered probe-side batch
/// 3. Concat them together according to `col_indices`, and return the result
///    (None if the result is empty)
///
/// Example:
/// build_side_batch:
/// a
/// ----
/// 1
/// 2
/// 3
///
/// # 0 index element in the build_side_batch (that is `1`) will be used
/// build_side_index: 0
///
/// probe_side_batch:
/// b
/// ----
/// 10
/// 20
/// 30
/// 40
///
/// # After applying it, only index 1 and 3 elements in probe_side_batch will be
/// # kept
/// probe_side_filter:
/// false
/// true
/// false
/// true
///
///
/// # Projections to the build/probe side batch, to construct the output batch
/// col_indices:
/// [(left, 0), (right, 0)]
///
/// build_side: left
///
/// ====
/// Result batch:
/// a b
/// ----
/// 1 20
/// 1 40
fn build_row_join_batch(
    output_schema: &Schema,
    build_side_batch: &RecordBatch,
    build_side_index: usize,
    probe_side_batch: &RecordBatch,
    probe_side_filter: Option<BooleanArray>,
    // See [`NLJStream`] struct's `column_indices` field for more detail
    col_indices: &[ColumnIndex],
    // If the build side is left or right, used to interpret the side information
    // in `col_indices`
    build_side: JoinSide,
) -> Result<Option<RecordBatch>> {
    debug_assert!(build_side != JoinSide::None);

    // TODO(perf): since the output might be projection of right batch, this
    // filtering step is more efficient to be done inside the column_index loop
    let filtered_probe_batch = if let Some(filter) = probe_side_filter {
        &filter_record_batch(probe_side_batch, &filter)?
    } else {
        probe_side_batch
    };

    if filtered_probe_batch.num_rows() == 0 {
        return Ok(None);
    }

    // Edge case: downstream operator does not require any columns from this NLJ,
    // so allow an empty projection.
    // Example:
    //  SELECT DISTINCT 32 AS col2
    //  FROM tab0 AS cor0
    //  LEFT OUTER JOIN tab2 AS cor1
    //  ON ( NULL ) IS NULL;
    if output_schema.fields.is_empty() {
        return Ok(Some(create_record_batch_with_empty_schema(
            Arc::new(output_schema.clone()),
            filtered_probe_batch.num_rows(),
        )?));
    }

    let mut columns: Vec<Arc<dyn Array>> =
        Vec::with_capacity(output_schema.fields().len());

    for column_index in col_indices {
        let array = if column_index.side == build_side {
            // Broadcast the single build-side row to match the filtered
            // probe-side batch length
            let original_left_array = build_side_batch.column(column_index.index);

            // Use `arrow::compute::take` directly for `List(Utf8View)` rather
            // than going through `ScalarValue::to_array_of_size()`, which
            // avoids some intermediate allocations.
            //
            // In other cases, `to_array_of_size()` is faster.
            match original_left_array.data_type() {
                DataType::List(field) | DataType::LargeList(field)
                    if field.data_type() == &DataType::Utf8View =>
                {
                    let indices_iter = std::iter::repeat_n(
                        build_side_index as u64,
                        filtered_probe_batch.num_rows(),
                    );
                    let indices_array = UInt64Array::from_iter_values(indices_iter);
                    take(original_left_array.as_ref(), &indices_array, None)?
                }
                _ => {
                    let scalar_value = ScalarValue::try_from_array(
                        original_left_array.as_ref(),
                        build_side_index,
                    )?;
                    scalar_value.to_array_of_size(filtered_probe_batch.num_rows())?
                }
            }
        } else {
            // Take the filtered probe-side column using compute::take
            Arc::clone(filtered_probe_batch.column(column_index.index))
        };

        columns.push(array);
    }

    Ok(Some(RecordBatch::try_new(
        Arc::new(output_schema.clone()),
        columns,
    )?))
}

/// Special case for `PlaceHolderRowExec`
/// Minimal example:  SELECT 1 WHERE EXISTS (SELECT 1);
//
/// # Return
/// If Some, that's the result batch
/// If None, it's not for this special case. Continue execution.
fn build_unmatched_batch_empty_schema(
    output_schema: &SchemaRef,
    batch_bitmap: &BooleanArray,
    // For left/right/full joins, it needs to fill nulls for another side
    join_type: JoinType,
) -> Result<Option<RecordBatch>> {
    let result_size = match join_type {
        JoinType::Left
        | JoinType::Right
        | JoinType::Full
        | JoinType::LeftAnti
        | JoinType::RightAnti => batch_bitmap.false_count(),
        JoinType::LeftSemi | JoinType::RightSemi => batch_bitmap.true_count(),
        JoinType::LeftMark | JoinType::RightMark => batch_bitmap.len(),
        _ => unreachable!(),
    };

    if output_schema.fields().is_empty() {
        Ok(Some(create_record_batch_with_empty_schema(
            Arc::clone(output_schema),
            result_size,
        )?))
    } else {
        Ok(None)
    }
}

/// Creates an empty RecordBatch with a specific row count.
/// This is useful for cases where we need a batch with the correct schema and row count
/// but no actual data columns (e.g., for constant filters).
fn create_record_batch_with_empty_schema(
    schema: SchemaRef,
    row_count: usize,
) -> Result<RecordBatch> {
    let options = RecordBatchOptions::new()
        .with_match_field_names(true)
        .with_row_count(Some(row_count));

    RecordBatch::try_new_with_options(schema, vec![], &options).map_err(|e| {
        internal_datafusion_err!("Failed to create empty record batch: {}", e)
    })
}

/// # Example:
/// batch:
/// a
/// ----
/// 1
/// 2
/// 3
///
/// batch_bitmap:
/// ----
/// false
/// true
/// false
///
/// another_side_schema:
/// [(b, bool), (c, int32)]
///
/// join_type: JoinType::Left
///
/// col_indices: ...(please refer to the comment in `NLJStream::column_indices``)
///
/// batch_side: right
///
/// # Walkthrough:
///
/// This executor is performing a right join, and the currently processed right
/// batch is as above. After joining it with all buffered left rows, the joined
/// entries are marked by the `batch_bitmap`.
/// This method will keep the unmatched indices on the batch side (right), and pad
/// the left side with nulls. The result would be:
///
/// b          c           a
/// ------------------------
/// Null(bool) Null(Int32) 1
/// Null(bool) Null(Int32) 3
fn build_unmatched_batch(
    output_schema: &SchemaRef,
    batch: &RecordBatch,
    batch_bitmap: BooleanArray,
    // For left/right/full joins, it needs to fill nulls for another side
    another_side_schema: &SchemaRef,
    col_indices: &[ColumnIndex],
    join_type: JoinType,
    batch_side: JoinSide,
) -> Result<Option<RecordBatch>> {
    // Should not call it for inner joins
    debug_assert_ne!(join_type, JoinType::Inner);
    debug_assert_ne!(batch_side, JoinSide::None);

    // Handle special case (see function comment)
    if let Some(batch) =
        build_unmatched_batch_empty_schema(output_schema, &batch_bitmap, join_type)?
    {
        return Ok(Some(batch));
    }

    match join_type {
        JoinType::Full | JoinType::Right | JoinType::Left => {
            if join_type == JoinType::Right {
                debug_assert_eq!(batch_side, JoinSide::Right);
            }
            if join_type == JoinType::Left {
                debug_assert_eq!(batch_side, JoinSide::Left);
            }

            // 1. Filter the batch with *flipped* bitmap
            // 2. Fill left side with nulls
            let flipped_bitmap = not(&batch_bitmap)?;

            // create a record batch, with left_schema, of only one row of all nulls
            let left_null_columns: Vec<Arc<dyn Array>> = another_side_schema
                .fields()
                .iter()
                .map(|field| new_null_array(field.data_type(), 1))
                .collect();

            // Hack: If the left schema is not nullable, the full join result
            // might contain null, this is only a temporary batch to construct
            // such full join result.
            let nullable_left_schema = Arc::new(Schema::new(
                another_side_schema
                    .fields()
                    .iter()
                    .map(|field| (**field).clone().with_nullable(true))
                    .collect::<Vec<_>>(),
            ));
            let left_null_batch = if nullable_left_schema.fields.is_empty() {
                // Left input can be an empty relation, in this case left relation
                // won't be used to construct the result batch (i.e. not in `col_indices`)
                create_record_batch_with_empty_schema(nullable_left_schema, 0)?
            } else {
                RecordBatch::try_new(nullable_left_schema, left_null_columns)?
            };

            debug_assert_ne!(batch_side, JoinSide::None);
            let opposite_side = batch_side.negate();

            build_row_join_batch(
                output_schema,
                &left_null_batch,
                0,
                batch,
                Some(flipped_bitmap),
                col_indices,
                opposite_side,
            )
        }
        JoinType::RightSemi
        | JoinType::RightAnti
        | JoinType::LeftSemi
        | JoinType::LeftAnti => {
            if matches!(join_type, JoinType::RightSemi | JoinType::RightAnti) {
                debug_assert_eq!(batch_side, JoinSide::Right);
            }
            if matches!(join_type, JoinType::LeftSemi | JoinType::LeftAnti) {
                debug_assert_eq!(batch_side, JoinSide::Left);
            }

            let bitmap = if matches!(join_type, JoinType::LeftSemi | JoinType::RightSemi)
            {
                batch_bitmap.clone()
            } else {
                not(&batch_bitmap)?
            };

            if !bitmap.has_true() {
                return Ok(None);
            }

            let mut columns: Vec<Arc<dyn Array>> =
                Vec::with_capacity(output_schema.fields().len());

            for column_index in col_indices {
                debug_assert!(column_index.side == batch_side);

                let col = batch.column(column_index.index);
                let filtered_col = filter(col, &bitmap)?;

                columns.push(filtered_col);
            }

            Ok(Some(RecordBatch::try_new(
                Arc::clone(output_schema),
                columns,
            )?))
        }
        JoinType::RightMark | JoinType::LeftMark => {
            if join_type == JoinType::RightMark {
                debug_assert_eq!(batch_side, JoinSide::Right);
            }
            if join_type == JoinType::LeftMark {
                debug_assert_eq!(batch_side, JoinSide::Left);
            }

            let mut columns: Vec<Arc<dyn Array>> =
                Vec::with_capacity(output_schema.fields().len());

            // Hack to deal with the borrow checker
            let mut right_batch_bitmap_opt = Some(batch_bitmap);

            for column_index in col_indices {
                if column_index.side == batch_side {
                    let col = batch.column(column_index.index);

                    columns.push(Arc::clone(col));
                } else if column_index.side == JoinSide::None {
                    let right_batch_bitmap = std::mem::take(&mut right_batch_bitmap_opt);
                    match right_batch_bitmap {
                        Some(right_batch_bitmap) => {
                            columns.push(Arc::new(right_batch_bitmap))
                        }
                        None => unreachable!("Should only be one mark column"),
                    }
                } else {
                    return internal_err!(
                        "Not possible to have this join side for RightMark join"
                    );
                }
            }

            Ok(Some(RecordBatch::try_new(
                Arc::clone(output_schema),
                columns,
            )?))
        }
        _ => internal_err!(
            "If batch is at right side, this function must be handling Full/Right/RightSemi/RightAnti/RightMark joins"
        ),
    }
}

#[cfg(test)]
pub(crate) mod tests {
    use super::*;
    use crate::test::{TestMemoryExec, assert_join_metrics};
    use crate::{
        common, expressions::Column, repartition::RepartitionExec, test::build_table_i32,
    };

    use arrow::compute::SortOptions;
    use arrow::datatypes::{DataType, Field};
    use datafusion_common::assert_contains;
    use datafusion_common::test_util::batches_to_sort_string;
    use datafusion_execution::runtime_env::RuntimeEnvBuilder;
    use datafusion_expr::Operator;
    use datafusion_physical_expr::expressions::{BinaryExpr, Literal};
    use datafusion_physical_expr::{Partitioning, PhysicalExpr};
    use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr};

    use insta::allow_duplicates;
    use insta::assert_snapshot;
    use rstest::rstest;

    fn build_table(
        a: (&str, &Vec<i32>),
        b: (&str, &Vec<i32>),
        c: (&str, &Vec<i32>),
        batch_size: Option<usize>,
        sorted_column_names: Vec<&str>,
    ) -> Arc<dyn ExecutionPlan> {
        let batch = build_table_i32(a, b, c);
        let schema = batch.schema();

        let batches = if let Some(batch_size) = batch_size {
            let num_batches = batch.num_rows().div_ceil(batch_size);
            (0..num_batches)
                .map(|i| {
                    let start = i * batch_size;
                    let remaining_rows = batch.num_rows() - start;
                    batch.slice(start, batch_size.min(remaining_rows))
                })
                .collect::<Vec<_>>()
        } else {
            vec![batch]
        };

        let mut sort_info = vec![];
        for name in sorted_column_names {
            let index = schema.index_of(name).unwrap();
            let sort_expr = PhysicalSortExpr::new(
                Arc::new(Column::new(name, index)),
                SortOptions::new(false, false),
            );
            sort_info.push(sort_expr);
        }
        let mut source = TestMemoryExec::try_new(&[batches], schema, None).unwrap();
        if let Some(ordering) = LexOrdering::new(sort_info) {
            source = source.try_with_sort_information(vec![ordering]).unwrap();
        }

        let source = Arc::new(source);
        Arc::new(TestMemoryExec::update_cache(&source))
    }

    fn build_left_table() -> Arc<dyn ExecutionPlan> {
        build_table(
            ("a1", &vec![5, 9, 11]),
            ("b1", &vec![5, 8, 8]),
            ("c1", &vec![50, 90, 110]),
            None,
            Vec::new(),
        )
    }

    fn build_right_table() -> Arc<dyn ExecutionPlan> {
        build_table(
            ("a2", &vec![12, 2, 10]),
            ("b2", &vec![10, 2, 10]),
            ("c2", &vec![40, 80, 100]),
            None,
            Vec::new(),
        )
    }

    fn prepare_join_filter() -> JoinFilter {
        let column_indices = vec![
            ColumnIndex {
                index: 1,
                side: JoinSide::Left,
            },
            ColumnIndex {
                index: 1,
                side: JoinSide::Right,
            },
        ];
        let intermediate_schema = Schema::new(vec![
            Field::new("x", DataType::Int32, true),
            Field::new("x", DataType::Int32, true),
        ]);
        // left.b1!=8
        let left_filter = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("x", 0)),
            Operator::NotEq,
            Arc::new(Literal::new(ScalarValue::Int32(Some(8)))),
        )) as Arc<dyn PhysicalExpr>;
        // right.b2!=10
        let right_filter = Arc::new(BinaryExpr::new(
            Arc::new(Column::new("x", 1)),
            Operator::NotEq,
            Arc::new(Literal::new(ScalarValue::Int32(Some(10)))),
        )) as Arc<dyn PhysicalExpr>;
        // filter = left.b1!=8 and right.b2!=10
        // after filter:
        // left table:
        // ("a1", &vec![5]),
        // ("b1", &vec![5]),
        // ("c1", &vec![50]),
        // right table:
        // ("a2", &vec![12, 2]),
        // ("b2", &vec![10, 2]),
        // ("c2", &vec![40, 80]),
        let filter_expression =
            Arc::new(BinaryExpr::new(left_filter, Operator::And, right_filter))
                as Arc<dyn PhysicalExpr>;

        JoinFilter::new(
            filter_expression,
            column_indices,
            Arc::new(intermediate_schema),
        )
    }

    pub(crate) async fn multi_partitioned_join_collect(
        left: Arc<dyn ExecutionPlan>,
        right: Arc<dyn ExecutionPlan>,
        join_type: &JoinType,
        join_filter: Option<JoinFilter>,
        context: Arc<TaskContext>,
    ) -> Result<(Vec<String>, Vec<RecordBatch>, MetricsSet)> {
        let partition_count = 4;

        // Redistributing right input
        let right = Arc::new(RepartitionExec::try_new(
            right,
            Partitioning::RoundRobinBatch(partition_count),
        )?) as Arc<dyn ExecutionPlan>;

        // Use the required distribution for nested loop join to test partition data
        let nested_loop_join =
            NestedLoopJoinExec::try_new(left, right, join_filter, join_type, None)?;
        let columns = columns(&nested_loop_join.schema());
        let mut batches = vec![];
        for i in 0..partition_count {
            let stream = nested_loop_join.execute(i, Arc::clone(&context))?;
            let more_batches = common::collect(stream).await?;
            batches.extend(
                more_batches
                    .into_iter()
                    .inspect(|b| {
                        assert!(b.num_rows() <= context.session_config().batch_size())
                    })
                    .filter(|b| b.num_rows() > 0)
                    .collect::<Vec<_>>(),
            );
        }

        let metrics = nested_loop_join.metrics().unwrap();

        Ok((columns, batches, metrics))
    }

    fn new_task_ctx(batch_size: usize) -> Arc<TaskContext> {
        let base = TaskContext::default();
        // limit max size of intermediate batch used in nlj to 1
        let cfg = base.session_config().clone().with_batch_size(batch_size);
        Arc::new(base.with_session_config(cfg))
    }

    #[rstest]
    #[tokio::test]
    async fn join_inner_with_filter(#[values(1, 2, 16)] batch_size: usize) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        dbg!(&batch_size);
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::Inner,
            Some(filter),
            task_ctx,
        )
        .await?;

        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+----+----+----+----+
        | a1 | b1 | c1 | a2 | b2 | c2 |
        +----+----+----+----+----+----+
        | 5  | 5  | 50 | 2  | 2  | 80 |
        +----+----+----+----+----+----+
        "));

        assert_join_metrics!(metrics, 1);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_left_with_filter(#[values(1, 2, 16)] batch_size: usize) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::Left,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+----+----+----+
        | a1 | b1 | c1  | a2 | b2 | c2 |
        +----+----+-----+----+----+----+
        | 11 | 8  | 110 |    |    |    |
        | 5  | 5  | 50  | 2  | 2  | 80 |
        | 9  | 8  | 90  |    |    |    |
        +----+----+-----+----+----+----+
        "));

        assert_join_metrics!(metrics, 3);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_right_with_filter(#[values(1, 2, 16)] batch_size: usize) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::Right,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+----+----+----+-----+
        | a1 | b1 | c1 | a2 | b2 | c2  |
        +----+----+----+----+----+-----+
        |    |    |    | 10 | 10 | 100 |
        |    |    |    | 12 | 10 | 40  |
        | 5  | 5  | 50 | 2  | 2  | 80  |
        +----+----+----+----+----+-----+
        "));

        assert_join_metrics!(metrics, 3);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_full_with_filter(#[values(1, 2, 16)] batch_size: usize) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::Full,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+----+----+-----+
        | a1 | b1 | c1  | a2 | b2 | c2  |
        +----+----+-----+----+----+-----+
        |    |    |     | 10 | 10 | 100 |
        |    |    |     | 12 | 10 | 40  |
        | 11 | 8  | 110 |    |    |     |
        | 5  | 5  | 50  | 2  | 2  | 80  |
        | 9  | 8  | 90  |    |    |     |
        +----+----+-----+----+----+-----+
        "));

        assert_join_metrics!(metrics, 5);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_left_semi_with_filter(
        #[values(1, 2, 16)] batch_size: usize,
    ) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::LeftSemi,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a1", "b1", "c1"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+----+
        | a1 | b1 | c1 |
        +----+----+----+
        | 5  | 5  | 50 |
        +----+----+----+
        "));

        assert_join_metrics!(metrics, 1);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_left_anti_with_filter(
        #[values(1, 2, 16)] batch_size: usize,
    ) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::LeftAnti,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a1", "b1", "c1"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+
        | a1 | b1 | c1  |
        +----+----+-----+
        | 11 | 8  | 110 |
        | 9  | 8  | 90  |
        +----+----+-----+
        "));

        assert_join_metrics!(metrics, 2);

        Ok(())
    }

    #[tokio::test]
    async fn join_has_correct_stats() -> Result<()> {
        let left = build_left_table();
        let right = build_right_table();
        let nested_loop_join = NestedLoopJoinExec::try_new(
            left,
            right,
            None,
            &JoinType::Left,
            Some(vec![1, 2]),
        )?;
        let stats = nested_loop_join.partition_statistics(None)?;
        assert_eq!(
            nested_loop_join.schema().fields().len(),
            stats.column_statistics.len(),
        );
        assert_eq!(2, stats.column_statistics.len());
        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_right_semi_with_filter(
        #[values(1, 2, 16)] batch_size: usize,
    ) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::RightSemi,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a2", "b2", "c2"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+----+
        | a2 | b2 | c2 |
        +----+----+----+
        | 2  | 2  | 80 |
        +----+----+----+
        "));

        assert_join_metrics!(metrics, 1);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_right_anti_with_filter(
        #[values(1, 2, 16)] batch_size: usize,
    ) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::RightAnti,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a2", "b2", "c2"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+
        | a2 | b2 | c2  |
        +----+----+-----+
        | 10 | 10 | 100 |
        | 12 | 10 | 40  |
        +----+----+-----+
        "));

        assert_join_metrics!(metrics, 2);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_left_mark_with_filter(
        #[values(1, 2, 16)] batch_size: usize,
    ) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::LeftMark,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a1", "b1", "c1", "mark"]);
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+-------+
        | a1 | b1 | c1  | mark  |
        +----+----+-----+-------+
        | 11 | 8  | 110 | false |
        | 5  | 5  | 50  | true  |
        | 9  | 8  | 90  | false |
        +----+----+-----+-------+
        "));

        assert_join_metrics!(metrics, 3);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn join_right_mark_with_filter(
        #[values(1, 2, 16)] batch_size: usize,
    ) -> Result<()> {
        let task_ctx = new_task_ctx(batch_size);
        let left = build_left_table();
        let right = build_right_table();

        let filter = prepare_join_filter();
        let (columns, batches, metrics) = multi_partitioned_join_collect(
            left,
            right,
            &JoinType::RightMark,
            Some(filter),
            task_ctx,
        )
        .await?;
        assert_eq!(columns, vec!["a2", "b2", "c2", "mark"]);

        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+-------+
        | a2 | b2 | c2  | mark  |
        +----+----+-----+-------+
        | 10 | 10 | 100 | false |
        | 12 | 10 | 40  | false |
        | 2  | 2  | 80  | true  |
        +----+----+-----+-------+
        "));

        assert_join_metrics!(metrics, 3);

        Ok(())
    }

    #[tokio::test]
    async fn test_overallocation() -> Result<()> {
        let left = build_table(
            ("a1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
            ("b1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
            ("c1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]),
            None,
            Vec::new(),
        );
        let right = build_table(
            ("a2", &vec![10, 11]),
            ("b2", &vec![12, 13]),
            ("c2", &vec![14, 15]),
            None,
            Vec::new(),
        );
        let filter = prepare_join_filter();

        // Join types that support memory-limited fallback should succeed
        // even under tight memory limits (they spill to disk instead of OOM).
        let fallback_join_types = vec![
            JoinType::Inner,
            JoinType::Left,
            JoinType::LeftSemi,
            JoinType::LeftAnti,
            JoinType::LeftMark,
            JoinType::Right,
            JoinType::RightSemi,
            JoinType::RightAnti,
            JoinType::RightMark,
        ];

        for join_type in &fallback_join_types {
            let runtime = RuntimeEnvBuilder::new()
                .with_memory_limit(100, 1.0)
                .build_arc()?;
            let task_ctx = TaskContext::default().with_runtime(runtime);
            let task_ctx = Arc::new(task_ctx);

            // Should succeed via spill fallback, not OOM
            let _result = multi_partitioned_join_collect(
                Arc::clone(&left),
                Arc::clone(&right),
                join_type,
                Some(filter.clone()),
                task_ctx,
            )
            .await?;
        }

        // FULL JOIN with multiple right partitions is intentionally not
        // supported in the fallback path yet (cross-partition left-bitmap
        // coordination is missing). It should still OOM under tight memory.
        let runtime = RuntimeEnvBuilder::new()
            .with_memory_limit(100, 1.0)
            .build_arc()?;
        let task_ctx = TaskContext::default().with_runtime(runtime);
        let task_ctx = Arc::new(task_ctx);
        let err = multi_partitioned_join_collect(
            Arc::clone(&left),
            Arc::clone(&right),
            &JoinType::Full,
            Some(filter.clone()),
            task_ctx,
        )
        .await
        .unwrap_err();
        assert_contains!(err.to_string(), "Resources exhausted");

        Ok(())
    }

    /// Returns the column names on the schema
    fn columns(schema: &Schema) -> Vec<String> {
        schema.fields().iter().map(|f| f.name().clone()).collect()
    }

    // ========================================================================
    // Memory-limited execution tests
    // ========================================================================

    /// Helper to run a NLJ using partition 0 and collect results + metrics.
    async fn join_collect(
        left: Arc<dyn ExecutionPlan>,
        right: Arc<dyn ExecutionPlan>,
        join_type: &JoinType,
        join_filter: Option<JoinFilter>,
        context: Arc<TaskContext>,
    ) -> Result<(Vec<String>, Vec<RecordBatch>, MetricsSet)> {
        let nested_loop_join =
            NestedLoopJoinExec::try_new(left, right, join_filter, join_type, None)?;
        let columns = columns(&nested_loop_join.schema());
        let stream = nested_loop_join.execute(0, context)?;
        let batches: Vec<RecordBatch> = common::collect(stream)
            .await?
            .into_iter()
            .filter(|b| b.num_rows() > 0)
            .collect();
        let metrics = nested_loop_join.metrics().unwrap();
        Ok((columns, batches, metrics))
    }

    /// Create a TaskContext with tight memory limit and disk spilling enabled.
    fn task_ctx_with_memory_limit(
        memory_limit: usize,
        batch_size: usize,
    ) -> Result<Arc<TaskContext>> {
        let runtime = RuntimeEnvBuilder::new()
            .with_memory_limit(memory_limit, 1.0)
            .build_arc()?;
        let cfg = TaskContext::default()
            .session_config()
            .clone()
            .with_batch_size(batch_size);
        let task_ctx = TaskContext::default()
            .with_runtime(runtime)
            .with_session_config(cfg);
        Ok(Arc::new(task_ctx))
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_inner_join() -> Result<()> {
        // Use a very small memory limit to force OOM → fallback to spill.
        let task_ctx = task_ctx_with_memory_limit(50, 16)?;
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (columns, batches, metrics) =
            join_collect(left, right, &JoinType::Inner, Some(filter), task_ctx).await?;

        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);

        // Verify spill actually occurred (memory-limited path was taken)
        assert!(
            metrics.spill_count().unwrap_or(0) > 0,
            "Expected spilling to occur under tight memory limit"
        );

        // Result should be identical to the non-memory-limited case
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+----+----+----+----+
        | a1 | b1 | c1 | a2 | b2 | c2 |
        +----+----+----+----+----+----+
        | 5  | 5  | 50 | 2  | 2  | 80 |
        +----+----+----+----+----+----+
        "));
        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_left_join() -> Result<()> {
        let task_ctx = task_ctx_with_memory_limit(50, 16)?;
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (columns, batches, metrics) =
            join_collect(left, right, &JoinType::Left, Some(filter), task_ctx).await?;

        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);

        // Verify spill actually occurred
        assert!(
            metrics.spill_count().unwrap_or(0) > 0,
            "Expected spilling to occur under tight memory limit"
        );

        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+----+----+----+
        | a1 | b1 | c1  | a2 | b2 | c2 |
        +----+----+-----+----+----+----+
        | 11 | 8  | 110 |    |    |    |
        | 5  | 5  | 50  | 2  | 2  | 80 |
        | 9  | 8  | 90  |    |    |    |
        +----+----+-----+----+----+----+
        "));
        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_fits_in_memory_no_spill() -> Result<()> {
        // Use a large memory limit — everything fits, no spilling needed.
        let task_ctx = task_ctx_with_memory_limit(10_000_000, 16)?;
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (columns, batches, metrics) =
            join_collect(left, right, &JoinType::Inner, Some(filter), task_ctx).await?;

        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);

        // Verify no spilling occurred (standard OnceFut path was used)
        assert_eq!(
            metrics.spill_count().unwrap_or(0),
            0,
            "Expected no spilling with generous memory limit"
        );

        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+----+----+----+----+
        | a1 | b1 | c1 | a2 | b2 | c2 |
        +----+----+----+----+----+----+
        | 5  | 5  | 50 | 2  | 2  | 80 |
        +----+----+----+----+----+----+
        "));
        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_empty_inputs() -> Result<()> {
        let task_ctx = task_ctx_with_memory_limit(50, 16)?;

        // Empty left table
        let empty_left = build_table(
            ("a1", &vec![]),
            ("b1", &vec![]),
            ("c1", &vec![]),
            None,
            Vec::new(),
        );
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (_columns, batches, _metrics) =
            join_collect(empty_left, right, &JoinType::Inner, Some(filter), task_ctx)
                .await?;
        assert!(batches.is_empty() || batches.iter().all(|b| b.num_rows() == 0));

        // Empty right table
        let task_ctx2 = task_ctx_with_memory_limit(50, 16)?;
        let left = build_left_table();
        let empty_right = build_table(
            ("a2", &vec![]),
            ("b2", &vec![]),
            ("c2", &vec![]),
            None,
            Vec::new(),
        );
        let filter2 = prepare_join_filter();

        let (_columns, batches, _metrics) = join_collect(
            left,
            empty_right,
            &JoinType::Inner,
            Some(filter2),
            task_ctx2,
        )
        .await?;
        assert!(batches.is_empty() || batches.iter().all(|b| b.num_rows() == 0));

        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_no_disk_falls_back_to_oom() -> Result<()> {
        // When disk is disabled, fallback is not possible and OOM should occur.
        use datafusion_execution::disk_manager::{DiskManagerBuilder, DiskManagerMode};

        let runtime = RuntimeEnvBuilder::new()
            .with_memory_limit(100, 1.0)
            .with_disk_manager_builder(
                DiskManagerBuilder::default().with_mode(DiskManagerMode::Disabled),
            )
            .build_arc()?;
        let task_ctx = Arc::new(TaskContext::default().with_runtime(runtime));

        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let err = join_collect(left, right, &JoinType::Inner, Some(filter), task_ctx)
            .await
            .unwrap_err();

        assert_contains!(err.to_string(), "Resources exhausted");
        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_right_join() -> Result<()> {
        let task_ctx = task_ctx_with_memory_limit(50, 16)?;
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (columns, batches, metrics) =
            join_collect(left, right, &JoinType::Right, Some(filter), task_ctx).await?;

        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);

        // Verify spill actually occurred
        assert!(
            metrics.spill_count().unwrap_or(0) > 0,
            "Expected spilling to occur under tight memory limit"
        );

        // Right join: all right rows appear. Unmatched right rows get NULLs on left.
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+----+----+----+-----+
        | a1 | b1 | c1 | a2 | b2 | c2  |
        +----+----+----+----+----+-----+
        |    |    |    | 10 | 10 | 100 |
        |    |    |    | 12 | 10 | 40  |
        | 5  | 5  | 50 | 2  | 2  | 80  |
        +----+----+----+----+----+-----+
        "));
        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_full_join() -> Result<()> {
        let task_ctx = task_ctx_with_memory_limit(50, 16)?;
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (columns, batches, metrics) =
            join_collect(left, right, &JoinType::Full, Some(filter), task_ctx).await?;

        assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b2", "c2"]);

        // Verify spill actually occurred
        assert!(
            metrics.spill_count().unwrap_or(0) > 0,
            "Expected spilling to occur under tight memory limit"
        );

        // Full join: unmatched from both sides appear with NULL padding.
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+----+----+-----+
        | a1 | b1 | c1  | a2 | b2 | c2  |
        +----+----+-----+----+----+-----+
        |    |    |     | 10 | 10 | 100 |
        |    |    |     | 12 | 10 | 40  |
        | 11 | 8  | 110 |    |    |     |
        | 5  | 5  | 50  | 2  | 2  | 80  |
        | 9  | 8  | 90  |    |    |     |
        +----+----+-----+----+----+-----+
        "));
        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_right_semi_join() -> Result<()> {
        let task_ctx = task_ctx_with_memory_limit(50, 16)?;
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (columns, batches, metrics) =
            join_collect(left, right, &JoinType::RightSemi, Some(filter), task_ctx)
                .await?;

        assert_eq!(columns, vec!["a2", "b2", "c2"]);

        assert!(
            metrics.spill_count().unwrap_or(0) > 0,
            "Expected spilling to occur under tight memory limit"
        );

        // Right semi: only right rows that matched at least one left row.
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+----+
        | a2 | b2 | c2 |
        +----+----+----+
        | 2  | 2  | 80 |
        +----+----+----+
        "));
        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_right_anti_join() -> Result<()> {
        let task_ctx = task_ctx_with_memory_limit(50, 16)?;
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (columns, batches, metrics) =
            join_collect(left, right, &JoinType::RightAnti, Some(filter), task_ctx)
                .await?;

        assert_eq!(columns, vec!["a2", "b2", "c2"]);

        assert!(
            metrics.spill_count().unwrap_or(0) > 0,
            "Expected spilling to occur under tight memory limit"
        );

        // Right anti: right rows that did NOT match any left row.
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+
        | a2 | b2 | c2  |
        +----+----+-----+
        | 10 | 10 | 100 |
        | 12 | 10 | 40  |
        +----+----+-----+
        "));
        Ok(())
    }

    #[tokio::test]
    async fn test_nlj_memory_limited_right_mark_join() -> Result<()> {
        let task_ctx = task_ctx_with_memory_limit(50, 16)?;
        let left = build_left_table();
        let right = build_right_table();
        let filter = prepare_join_filter();

        let (columns, batches, metrics) =
            join_collect(left, right, &JoinType::RightMark, Some(filter), task_ctx)
                .await?;

        assert_eq!(columns, vec!["a2", "b2", "c2", "mark"]);

        assert!(
            metrics.spill_count().unwrap_or(0) > 0,
            "Expected spilling to occur under tight memory limit"
        );

        // Right mark: all right rows with a bool column indicating match.
        allow_duplicates!(assert_snapshot!(batches_to_sort_string(&batches), @r"
        +----+----+-----+-------+
        | a2 | b2 | c2  | mark  |
        +----+----+-----+-------+
        | 10 | 10 | 100 | false |
        | 12 | 10 | 40  | false |
        | 2  | 2  | 80  | true  |
        +----+----+-----+-------+
        "));
        Ok(())
    }
}