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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! This file implements the symmetric hash join algorithm with range-based
//! data pruning to join two (potentially infinite) streams.
//!
//! A [SymmetricHashJoinExec] plan takes two children plan (with appropriate
//! output ordering) and produces the join output according to the given join
//! type and other options.
//!
//! This plan uses the [OneSideHashJoiner] object to facilitate join calculations
//! for both its children.
use std::collections::{HashMap, VecDeque};
use std::fmt;
use std::fmt::{Debug, Formatter};
use std::sync::Arc;
use std::task::Poll;
use std::vec;
use std::{any::Any, usize};
use ahash::RandomState;
use arrow::array::{
ArrowPrimitiveType, BooleanBufferBuilder, NativeAdapter, PrimitiveArray,
PrimitiveBuilder,
};
use arrow::compute::concat_batches;
use arrow::datatypes::{ArrowNativeType, Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use arrow_array::builder::{UInt32BufferBuilder, UInt64BufferBuilder};
use arrow_array::{UInt32Array, UInt64Array};
use datafusion_physical_expr::hash_utils::create_hashes;
use datafusion_physical_expr::PhysicalExpr;
use futures::stream::{select, BoxStream};
use futures::{Stream, StreamExt};
use hashbrown::HashSet;
use parking_lot::Mutex;
use smallvec::smallvec;
use datafusion_common::{utils::bisect, ScalarValue};
use datafusion_execution::memory_pool::MemoryConsumer;
use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound};
use crate::physical_plan::common::SharedMemoryReservation;
use crate::physical_plan::joins::hash_join_utils::convert_sort_expr_with_filter_schema;
use crate::physical_plan::{
expressions::Column,
expressions::PhysicalSortExpr,
joins::{
hash_join_utils::{build_filter_input_order, SortedFilterExpr},
utils::{
build_batch_from_indices, build_join_schema, check_join_is_valid,
combine_join_equivalence_properties, partitioned_join_output_partitioning,
ColumnIndex, JoinFilter, JoinOn, JoinSide,
},
},
metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet},
DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning,
RecordBatchStream, SendableRecordBatchStream, Statistics,
};
use datafusion_common::JoinType;
use datafusion_common::{DataFusionError, Result};
use datafusion_execution::TaskContext;
use super::hash_join::equal_rows;
use super::hash_join_utils::SymmetricJoinHashMap;
use super::utils::apply_join_filter_to_indices;
const HASHMAP_SHRINK_SCALE_FACTOR: usize = 4;
/// A symmetric hash join with range conditions is when both streams are hashed on the
/// join key and the resulting hash tables are used to join the streams.
/// The join is considered symmetric because the hash table is built on the join keys from both
/// streams, and the matching of rows is based on the values of the join keys in both streams.
/// This type of join is efficient in streaming context as it allows for fast lookups in the hash
/// table, rather than having to scan through one or both of the streams to find matching rows, also it
/// only considers the elements from the stream that fall within a certain sliding window (w/ range conditions),
/// making it more efficient and less likely to store stale data. This enables operating on unbounded streaming
/// data without any memory issues.
///
/// For each input stream, create a hash table.
/// - For each new [RecordBatch] in build side, hash and insert into inputs hash table. Update offsets.
/// - Test if input is equal to a predefined set of other inputs.
/// - If so record the visited rows. If the matched row results must be produced (INNER, LEFT), output the [RecordBatch].
/// - Try to prune other side (probe) with new [RecordBatch].
/// - If the join type indicates that the unmatched rows results must be produced (LEFT, FULL etc.),
/// output the [RecordBatch] when a pruning happens or at the end of the data.
///
///
/// ``` text
/// +-------------------------+
/// | |
/// left stream ---------| Left OneSideHashJoiner |---+
/// | | |
/// +-------------------------+ |
/// |
/// |--------- Joined output
/// |
/// +-------------------------+ |
/// | | |
/// right stream ---------| Right OneSideHashJoiner |---+
/// | |
/// +-------------------------+
///
/// Prune build side when the new RecordBatch comes to the probe side. We utilize interval arithmetic
/// on JoinFilter's sorted PhysicalExprs to calculate the joinable range.
///
///
/// PROBE SIDE BUILD SIDE
/// BUFFER BUFFER
/// +-------------+ +------------+
/// | | | | Unjoinable
/// | | | | Range
/// | | | |
/// | | |---------------------------------
/// | | | | |
/// | | | | |
/// | | / | |
/// | | | | |
/// | | | | |
/// | | | | |
/// | | | | |
/// | | | | | Joinable
/// | |/ | | Range
/// | || | |
/// |+-----------+|| | |
/// || Record || | |
/// || Batch || | |
/// |+-----------+|| | |
/// +-------------+\ +------------+
/// |
/// \
/// |---------------------------------
///
/// This happens when range conditions are provided on sorted columns. E.g.
///
/// SELECT * FROM left_table, right_table
/// ON
/// left_key = right_key AND
/// left_time > right_time - INTERVAL 12 MINUTES AND left_time < right_time + INTERVAL 2 HOUR
///
/// or
/// SELECT * FROM left_table, right_table
/// ON
/// left_key = right_key AND
/// left_sorted > right_sorted - 3 AND left_sorted < right_sorted + 10
///
/// For general purpose, in the second scenario, when the new data comes to probe side, the conditions can be used to
/// determine a specific threshold for discarding rows from the inner buffer. For example, if the sort order the
/// two columns ("left_sorted" and "right_sorted") are ascending (it can be different in another scenarios)
/// and the join condition is "left_sorted > right_sorted - 3" and the latest value on the right input is 1234, meaning
/// that the left side buffer must only keep rows where "leftTime > rightTime - 3 > 1234 - 3 > 1231" ,
/// making the smallest value in 'left_sorted' 1231 and any rows below (since ascending)
/// than that can be dropped from the inner buffer.
/// ```
#[derive(Debug)]
pub struct SymmetricHashJoinExec {
/// Left side stream
pub(crate) left: Arc<dyn ExecutionPlan>,
/// Right side stream
pub(crate) right: Arc<dyn ExecutionPlan>,
/// Set of common columns used to join on
pub(crate) on: Vec<(Column, Column)>,
/// Filters applied when finding matching rows
pub(crate) filter: Option<JoinFilter>,
/// How the join is performed
pub(crate) join_type: JoinType,
/// Expression graph and `SortedFilterExpr`s for interval calculations
filter_state: Option<Arc<Mutex<IntervalCalculatorInnerState>>>,
/// The schema once the join is applied
schema: SchemaRef,
/// Shares the `RandomState` for the hashing algorithm
random_state: RandomState,
/// Execution metrics
metrics: ExecutionPlanMetricsSet,
/// Information of index and left / right placement of columns
column_indices: Vec<ColumnIndex>,
/// If null_equals_null is true, null == null else null != null
pub(crate) null_equals_null: bool,
}
struct IntervalCalculatorInnerState {
/// Expression graph for interval calculations
graph: Option<ExprIntervalGraph>,
sorted_exprs: Vec<Option<SortedFilterExpr>>,
calculated: bool,
}
impl Debug for IntervalCalculatorInnerState {
fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
write!(f, "Exprs({:?})", self.sorted_exprs)
}
}
#[derive(Debug)]
struct SymmetricHashJoinSideMetrics {
/// Number of batches consumed by this operator
input_batches: metrics::Count,
/// Number of rows consumed by this operator
input_rows: metrics::Count,
}
/// Metrics for HashJoinExec
#[derive(Debug)]
struct SymmetricHashJoinMetrics {
/// Number of left batches/rows consumed by this operator
left: SymmetricHashJoinSideMetrics,
/// Number of right batches/rows consumed by this operator
right: SymmetricHashJoinSideMetrics,
/// Memory used by sides in bytes
pub(crate) stream_memory_usage: metrics::Gauge,
/// Number of batches produced by this operator
output_batches: metrics::Count,
/// Number of rows produced by this operator
output_rows: metrics::Count,
}
impl SymmetricHashJoinMetrics {
pub fn new(partition: usize, metrics: &ExecutionPlanMetricsSet) -> Self {
let input_batches =
MetricBuilder::new(metrics).counter("input_batches", partition);
let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition);
let left = SymmetricHashJoinSideMetrics {
input_batches,
input_rows,
};
let input_batches =
MetricBuilder::new(metrics).counter("input_batches", partition);
let input_rows = MetricBuilder::new(metrics).counter("input_rows", partition);
let right = SymmetricHashJoinSideMetrics {
input_batches,
input_rows,
};
let stream_memory_usage =
MetricBuilder::new(metrics).gauge("stream_memory_usage", partition);
let output_batches =
MetricBuilder::new(metrics).counter("output_batches", partition);
let output_rows = MetricBuilder::new(metrics).output_rows(partition);
Self {
left,
right,
output_batches,
stream_memory_usage,
output_rows,
}
}
}
impl SymmetricHashJoinExec {
/// Tries to create a new [SymmetricHashJoinExec].
/// # Error
/// This function errors when:
/// - It is not possible to join the left and right sides on keys `on`, or
/// - It fails to construct `SortedFilterExpr`s, or
/// - It fails to create the [ExprIntervalGraph].
pub fn try_new(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
filter: Option<JoinFilter>,
join_type: &JoinType,
null_equals_null: bool,
) -> Result<Self> {
let left_schema = left.schema();
let right_schema = right.schema();
// Error out if no "on" contraints are given:
if on.is_empty() {
return Err(DataFusionError::Plan(
"On constraints in SymmetricHashJoinExec should be non-empty".to_string(),
));
}
// Check if the join is valid with the given on constraints:
check_join_is_valid(&left_schema, &right_schema, &on)?;
// Build the join schema from the left and right schemas:
let (schema, column_indices) =
build_join_schema(&left_schema, &right_schema, join_type);
// Initialize the random state for the join operation:
let random_state = RandomState::with_seeds(0, 0, 0, 0);
let filter_state = if filter.is_some() {
let inner_state = IntervalCalculatorInnerState {
graph: None,
sorted_exprs: vec![],
calculated: false,
};
Some(Arc::new(Mutex::new(inner_state)))
} else {
None
};
Ok(SymmetricHashJoinExec {
left,
right,
on,
filter,
join_type: *join_type,
filter_state,
schema: Arc::new(schema),
random_state,
metrics: ExecutionPlanMetricsSet::new(),
column_indices,
null_equals_null,
})
}
/// left stream
pub fn left(&self) -> &Arc<dyn ExecutionPlan> {
&self.left
}
/// right stream
pub fn right(&self) -> &Arc<dyn ExecutionPlan> {
&self.right
}
/// Set of common columns used to join on
pub fn on(&self) -> &[(Column, Column)] {
&self.on
}
/// 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
}
/// Get null_equals_null
pub fn null_equals_null(&self) -> bool {
self.null_equals_null
}
/// Check if order information covers every column in the filter expression.
pub fn check_if_order_information_available(&self) -> Result<bool> {
if let Some(filter) = self.filter() {
let left = self.left();
if let Some(left_ordering) = left.output_ordering() {
let right = self.right();
if let Some(right_ordering) = right.output_ordering() {
let left_convertible = convert_sort_expr_with_filter_schema(
&JoinSide::Left,
filter,
&left.schema(),
&left_ordering[0],
)?
.is_some();
let right_convertible = convert_sort_expr_with_filter_schema(
&JoinSide::Right,
filter,
&right.schema(),
&right_ordering[0],
)?
.is_some();
return Ok(left_convertible && right_convertible);
}
}
}
Ok(false)
}
}
impl ExecutionPlan for SymmetricHashJoinExec {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
Ok(children.iter().any(|u| *u))
}
fn benefits_from_input_partitioning(&self) -> bool {
false
}
fn required_input_distribution(&self) -> Vec<Distribution> {
let (left_expr, right_expr) = self
.on
.iter()
.map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _))
.unzip();
// TODO: This will change when we extend collected executions.
vec![
Distribution::HashPartitioned(left_expr),
Distribution::HashPartitioned(right_expr),
]
}
fn output_partitioning(&self) -> Partitioning {
let left_columns_len = self.left.schema().fields.len();
partitioned_join_output_partitioning(
self.join_type,
self.left.output_partitioning(),
self.right.output_partitioning(),
left_columns_len,
)
}
// TODO: Output ordering might be kept for some cases.
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
None
}
fn equivalence_properties(&self) -> EquivalenceProperties {
let left_columns_len = self.left.schema().fields.len();
combine_join_equivalence_properties(
self.join_type,
self.left.equivalence_properties(),
self.right.equivalence_properties(),
left_columns_len,
self.on(),
self.schema(),
)
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
vec![self.left.clone(), self.right.clone()]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(SymmetricHashJoinExec::try_new(
children[0].clone(),
children[1].clone(),
self.on.clone(),
self.filter.clone(),
&self.join_type,
self.null_equals_null,
)?))
}
fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> 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 on = self
.on
.iter()
.map(|(c1, c2)| format!("({}, {})", c1, c2))
.collect::<Vec<String>>()
.join(", ");
write!(
f,
"SymmetricHashJoinExec: join_type={:?}, on=[{}]{}",
self.join_type, on, display_filter
)
}
}
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn statistics(&self) -> Statistics {
// TODO stats: it is not possible in general to know the output size of joins
Statistics::default()
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
let left_partitions = self.left.output_partitioning().partition_count();
let right_partitions = self.right.output_partitioning().partition_count();
if left_partitions != right_partitions {
return Err(DataFusionError::Internal(format!(
"Invalid SymmetricHashJoinExec, partition count mismatch {left_partitions}!={right_partitions},\
consider using RepartitionExec",
)));
}
// If `filter_state` and `filter` are both present, then calculate sorted filter expressions
// for both sides, and build an expression graph if one is not already built.
let (left_sorted_filter_expr, right_sorted_filter_expr, graph) =
match (&self.filter_state, &self.filter) {
(Some(interval_state), Some(filter)) => {
// Lock the mutex of the interval state:
let mut filter_state = interval_state.lock();
// If this is the first partition to be invoked, then we need to initialize our state
// (the expression graph for pruning, sorted filter expressions etc.)
if !filter_state.calculated {
// Interval calculations require each column to exhibit monotonicity
// independently. However, a `PhysicalSortExpr` object defines a
// lexicographical ordering, so we can only use their first elements.
// when deducing column monotonicities.
// TODO: Extend the `PhysicalSortExpr` mechanism to express independent
// (i.e. simultaneous) ordering properties of columns.
// Build sorted filter expressions for the left and right join side:
let join_sides = [JoinSide::Left, JoinSide::Right];
let children = [&self.left, &self.right];
for (join_side, child) in join_sides.iter().zip(children.iter()) {
let sorted_expr = child
.output_ordering()
.and_then(|orders| {
build_filter_input_order(
*join_side,
filter,
&child.schema(),
&orders[0],
)
.transpose()
})
.transpose()?;
filter_state.sorted_exprs.push(sorted_expr);
}
// Collect available sorted filter expressions:
let sorted_exprs_size = filter_state.sorted_exprs.len();
let mut sorted_exprs = filter_state
.sorted_exprs
.iter_mut()
.flatten()
.collect::<Vec<_>>();
// Create the expression graph if we can create sorted filter expressions for both children:
filter_state.graph = if sorted_exprs.len() == sorted_exprs_size {
let mut graph =
ExprIntervalGraph::try_new(filter.expression().clone())?;
// Gather filter expressions:
let filter_exprs = sorted_exprs
.iter()
.map(|sorted_expr| sorted_expr.filter_expr().clone())
.collect::<Vec<_>>();
// Gather node indices of converted filter expressions in `SortedFilterExpr`s
// using the filter columns vector:
let child_node_indices =
graph.gather_node_indices(&filter_exprs);
// Update SortedFilterExpr instances with the corresponding node indices:
for (sorted_expr, (_, index)) in
sorted_exprs.iter_mut().zip(child_node_indices.iter())
{
sorted_expr.set_node_index(*index);
}
Some(graph)
} else {
None
};
filter_state.calculated = true;
}
// Return the sorted filter expressions for both sides along with the expression graph:
(
filter_state.sorted_exprs[0].clone(),
filter_state.sorted_exprs[1].clone(),
filter_state.graph.as_ref().cloned(),
)
}
// If `filter_state` or `filter` is not present, then return None for all three values:
(_, _) => (None, None, None),
};
let on_left = self.on.iter().map(|on| on.0.clone()).collect::<Vec<_>>();
let on_right = self.on.iter().map(|on| on.1.clone()).collect::<Vec<_>>();
let left_side_joiner =
OneSideHashJoiner::new(JoinSide::Left, on_left, self.left.schema());
let right_side_joiner =
OneSideHashJoiner::new(JoinSide::Right, on_right, self.right.schema());
let left_stream = self
.left
.execute(partition, context.clone())?
.map(|val| (JoinSide::Left, val));
let right_stream = self
.right
.execute(partition, context.clone())?
.map(|val| (JoinSide::Right, val));
// This function will attempt to pull items from both streams.
// Each stream will be polled in a round-robin fashion, and whenever a stream is
// ready to yield an item that item is yielded.
// After one of the two input streams completes, the remaining one will be polled exclusively.
// The returned stream completes when both input streams have completed.
let input_stream = select(left_stream, right_stream).boxed();
let reservation = Arc::new(Mutex::new(
MemoryConsumer::new(format!("SymmetricHashJoinStream[{partition}]"))
.register(context.memory_pool()),
));
if let Some(g) = graph.as_ref() {
reservation.lock().try_grow(g.size())?;
}
Ok(Box::pin(SymmetricHashJoinStream {
input_stream,
schema: self.schema(),
filter: self.filter.clone(),
join_type: self.join_type,
random_state: self.random_state.clone(),
left: left_side_joiner,
right: right_side_joiner,
column_indices: self.column_indices.clone(),
metrics: SymmetricHashJoinMetrics::new(partition, &self.metrics),
graph,
left_sorted_filter_expr,
right_sorted_filter_expr,
null_equals_null: self.null_equals_null,
final_result: false,
reservation,
}))
}
}
/// A stream that issues [RecordBatch]es as they arrive from the right of the join.
struct SymmetricHashJoinStream {
/// Input stream
input_stream: BoxStream<'static, (JoinSide, Result<RecordBatch>)>,
/// Input schema
schema: Arc<Schema>,
/// join filter
filter: Option<JoinFilter>,
/// type of the join
join_type: JoinType,
// left hash joiner
left: OneSideHashJoiner,
/// right hash joiner
right: OneSideHashJoiner,
/// Information of index and left / right placement of columns
column_indices: Vec<ColumnIndex>,
// Expression graph for range pruning.
graph: Option<ExprIntervalGraph>,
// Left globally sorted filter expr
left_sorted_filter_expr: Option<SortedFilterExpr>,
// Right globally sorted filter expr
right_sorted_filter_expr: Option<SortedFilterExpr>,
/// Random state used for hashing initialization
random_state: RandomState,
/// If null_equals_null is true, null == null else null != null
null_equals_null: bool,
/// Metrics
metrics: SymmetricHashJoinMetrics,
/// Memory reservation
reservation: SharedMemoryReservation,
/// Flag indicating whether there is nothing to process anymore
final_result: bool,
}
impl RecordBatchStream for SymmetricHashJoinStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
impl Stream for SymmetricHashJoinStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: std::pin::Pin<&mut Self>,
cx: &mut std::task::Context<'_>,
) -> Poll<Option<Self::Item>> {
self.poll_next_impl(cx)
}
}
fn prune_hash_values(
prune_length: usize,
hashmap: &mut SymmetricJoinHashMap,
row_hash_values: &mut VecDeque<u64>,
offset: u64,
) -> Result<()> {
// Create a (hash)-(row number set) map
let mut hash_value_map: HashMap<u64, HashSet<u64>> = HashMap::new();
for index in 0..prune_length {
let hash_value = row_hash_values.pop_front().unwrap();
if let Some(set) = hash_value_map.get_mut(&hash_value) {
set.insert(offset + index as u64);
} else {
let mut set = HashSet::new();
set.insert(offset + index as u64);
hash_value_map.insert(hash_value, set);
}
}
for (hash_value, index_set) in hash_value_map.iter() {
if let Some((_, separation_chain)) = hashmap
.0
.get_mut(*hash_value, |(hash, _)| hash_value == hash)
{
separation_chain.retain(|n| !index_set.contains(n));
if separation_chain.is_empty() {
hashmap
.0
.remove_entry(*hash_value, |(hash, _)| hash_value == hash);
}
}
}
hashmap.shrink_if_necessary(HASHMAP_SHRINK_SCALE_FACTOR);
Ok(())
}
/// Calculate the filter expression intervals.
///
/// This function updates the `interval` field of each `SortedFilterExpr` based
/// on the first or the last value of the expression in `build_input_buffer`
/// and `probe_batch`.
///
/// # Arguments
///
/// * `build_input_buffer` - The [RecordBatch] on the build side of the join.
/// * `build_sorted_filter_expr` - Build side [SortedFilterExpr] to update.
/// * `probe_batch` - The `RecordBatch` on the probe side of the join.
/// * `probe_sorted_filter_expr` - Probe side `SortedFilterExpr` to update.
///
/// ### Note
/// ```text
///
/// Interval arithmetic is used to calculate viable join ranges for build-side
/// pruning. This is done by first creating an interval for join filter values in
/// the build side of the join, which spans [-∞, FV] or [FV, ∞] depending on the
/// ordering (descending/ascending) of the filter expression. Here, FV denotes the
/// first value on the build side. This range is then compared with the probe side
/// interval, which either spans [-∞, LV] or [LV, ∞] depending on the ordering
/// (ascending/descending) of the probe side. Here, LV denotes the last value on
/// the probe side.
///
/// As a concrete example, consider the following query:
///
/// SELECT * FROM left_table, right_table
/// WHERE
/// left_key = right_key AND
/// a > b - 3 AND
/// a < b + 10
///
/// where columns "a" and "b" come from tables "left_table" and "right_table",
/// respectively. When a new `RecordBatch` arrives at the right side, the
/// condition a > b - 3 will possibly indicate a prunable range for the left
/// side. Conversely, when a new `RecordBatch` arrives at the left side, the
/// condition a < b + 10 will possibly indicate prunability for the right side.
/// Let’s inspect what happens when a new RecordBatch` arrives at the right
/// side (i.e. when the left side is the build side):
///
/// Build Probe
/// +-------+ +-------+
/// | a | z | | b | y |
/// |+--|--+| |+--|--+|
/// | 1 | 2 | | 4 | 3 |
/// |+--|--+| |+--|--+|
/// | 3 | 1 | | 4 | 3 |
/// |+--|--+| |+--|--+|
/// | 5 | 7 | | 6 | 1 |
/// |+--|--+| |+--|--+|
/// | 7 | 1 | | 6 | 3 |
/// +-------+ +-------+
///
/// In this case, the interval representing viable (i.e. joinable) values for
/// column "a" is [1, ∞], and the interval representing possible future values
/// for column "b" is [6, ∞]. With these intervals at hand, we next calculate
/// intervals for the whole filter expression and propagate join constraint by
/// traversing the expression graph.
/// ```
fn calculate_filter_expr_intervals(
build_input_buffer: &RecordBatch,
build_sorted_filter_expr: &mut SortedFilterExpr,
probe_batch: &RecordBatch,
probe_sorted_filter_expr: &mut SortedFilterExpr,
) -> Result<()> {
// If either build or probe side has no data, return early:
if build_input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 {
return Ok(());
}
// Calculate the interval for the build side filter expression (if present):
update_filter_expr_interval(
&build_input_buffer.slice(0, 1),
build_sorted_filter_expr,
)?;
// Calculate the interval for the probe side filter expression (if present):
update_filter_expr_interval(
&probe_batch.slice(probe_batch.num_rows() - 1, 1),
probe_sorted_filter_expr,
)
}
/// This is a subroutine of the function [`calculate_filter_expr_intervals`].
/// It constructs the current interval using the given `batch` and updates
/// the filter expression (i.e. `sorted_expr`) with this interval.
fn update_filter_expr_interval(
batch: &RecordBatch,
sorted_expr: &mut SortedFilterExpr,
) -> Result<()> {
// Evaluate the filter expression and convert the result to an array:
let array = sorted_expr
.origin_sorted_expr()
.expr
.evaluate(batch)?
.into_array(1);
// Convert the array to a ScalarValue:
let value = ScalarValue::try_from_array(&array, 0)?;
// Create a ScalarValue representing positive or negative infinity for the same data type:
let unbounded = IntervalBound::make_unbounded(value.get_datatype())?;
// Update the interval with lower and upper bounds based on the sort option:
let interval = if sorted_expr.origin_sorted_expr().options.descending {
Interval::new(unbounded, IntervalBound::new(value, false))
} else {
Interval::new(IntervalBound::new(value, false), unbounded)
};
// Set the calculated interval for the sorted filter expression:
sorted_expr.set_interval(interval);
Ok(())
}
/// Determine the pruning length for `buffer`.
///
/// This function evaluates the build side filter expression, converts the
/// result into an array and determines the pruning length by performing a
/// binary search on the array.
///
/// # Arguments
///
/// * `buffer`: The record batch to be pruned.
/// * `build_side_filter_expr`: The filter expression on the build side used
/// to determine the pruning length.
///
/// # Returns
///
/// A [Result] object that contains the pruning length. The function will return
/// an error if there is an issue evaluating the build side filter expression.
fn determine_prune_length(
buffer: &RecordBatch,
build_side_filter_expr: &SortedFilterExpr,
) -> Result<usize> {
let origin_sorted_expr = build_side_filter_expr.origin_sorted_expr();
let interval = build_side_filter_expr.interval();
// Evaluate the build side filter expression and convert it into an array
let batch_arr = origin_sorted_expr
.expr
.evaluate(buffer)?
.into_array(buffer.num_rows());
// Get the lower or upper interval based on the sort direction
let target = if origin_sorted_expr.options.descending {
interval.upper.value.clone()
} else {
interval.lower.value.clone()
};
// Perform binary search on the array to determine the length of the record batch to be pruned
bisect::<true>(&[batch_arr], &[target], &[origin_sorted_expr.options])
}
/// This method determines if the result of the join should be produced in the final step or not.
///
/// # Arguments
///
/// * `build_side` - Enum indicating the side of the join used as the build side.
/// * `join_type` - Enum indicating the type of join to be performed.
///
/// # Returns
///
/// A boolean indicating whether the result of the join should be produced in the final step or not.
/// The result will be true if the build side is JoinSide::Left and the join type is one of
/// JoinType::Left, JoinType::LeftAnti, JoinType::Full or JoinType::LeftSemi.
/// If the build side is JoinSide::Right, the result will be true if the join type
/// is one of JoinType::Right, JoinType::RightAnti, JoinType::Full, or JoinType::RightSemi.
fn need_to_produce_result_in_final(build_side: JoinSide, join_type: JoinType) -> bool {
if build_side == JoinSide::Left {
matches!(
join_type,
JoinType::Left | JoinType::LeftAnti | JoinType::Full | JoinType::LeftSemi
)
} else {
matches!(
join_type,
JoinType::Right | JoinType::RightAnti | JoinType::Full | JoinType::RightSemi
)
}
}
/// Get the anti join indices from the visited hash set.
///
/// This method returns the indices from the original input that were not present in the visited hash set.
///
/// # Arguments
///
/// * `prune_length` - The length of the pruned record batch.
/// * `deleted_offset` - The offset to the indices.
/// * `visited_rows` - The hash set of visited indices.
///
/// # Returns
///
/// A `PrimitiveArray` of the anti join indices.
fn get_anti_indices<T: ArrowPrimitiveType>(
prune_length: usize,
deleted_offset: usize,
visited_rows: &HashSet<usize>,
) -> PrimitiveArray<T>
where
NativeAdapter<T>: From<<T as ArrowPrimitiveType>::Native>,
{
let mut bitmap = BooleanBufferBuilder::new(prune_length);
bitmap.append_n(prune_length, false);
// mark the indices as true if they are present in the visited hash set
for v in 0..prune_length {
let row = v + deleted_offset;
bitmap.set_bit(v, visited_rows.contains(&row));
}
// get the anti index
(0..prune_length)
.filter_map(|idx| (!bitmap.get_bit(idx)).then_some(T::Native::from_usize(idx)))
.collect()
}
/// This method creates a boolean buffer from the visited rows hash set
/// and the indices of the pruned record batch slice.
///
/// It gets the indices from the original input that were present in the visited hash set.
///
/// # Arguments
///
/// * `prune_length` - The length of the pruned record batch.
/// * `deleted_offset` - The offset to the indices.
/// * `visited_rows` - The hash set of visited indices.
///
/// # Returns
///
/// A [PrimitiveArray] of the specified type T, containing the semi indices.
fn get_semi_indices<T: ArrowPrimitiveType>(
prune_length: usize,
deleted_offset: usize,
visited_rows: &HashSet<usize>,
) -> PrimitiveArray<T>
where
NativeAdapter<T>: From<<T as ArrowPrimitiveType>::Native>,
{
let mut bitmap = BooleanBufferBuilder::new(prune_length);
bitmap.append_n(prune_length, false);
// mark the indices as true if they are present in the visited hash set
(0..prune_length).for_each(|v| {
let row = &(v + deleted_offset);
bitmap.set_bit(v, visited_rows.contains(row));
});
// get the semi index
(0..prune_length)
.filter_map(|idx| (bitmap.get_bit(idx)).then_some(T::Native::from_usize(idx)))
.collect::<PrimitiveArray<T>>()
}
/// Records the visited indices from the input `PrimitiveArray` of type `T` into the given hash set `visited`.
/// This function will insert the indices (offset by `offset`) into the `visited` hash set.
///
/// # Arguments
///
/// * `visited` - A hash set to store the visited indices.
/// * `offset` - An offset to the indices in the `PrimitiveArray`.
/// * `indices` - The input `PrimitiveArray` of type `T` which stores the indices to be recorded.
///
fn record_visited_indices<T: ArrowPrimitiveType>(
visited: &mut HashSet<usize>,
offset: usize,
indices: &PrimitiveArray<T>,
) {
for i in indices.values() {
visited.insert(i.as_usize() + offset);
}
}
/// Calculate indices by join type.
///
/// This method returns a tuple of two arrays: build and probe indices.
/// The length of both arrays will be the same.
///
/// # Arguments
///
/// * `build_side`: Join side which defines the build side.
/// * `prune_length`: Length of the prune data.
/// * `visited_rows`: Hash set of visited rows of the build side.
/// * `deleted_offset`: Deleted offset of the build side.
/// * `join_type`: The type of join to be performed.
///
/// # Returns
///
/// A tuple of two arrays of primitive types representing the build and probe indices.
///
fn calculate_indices_by_join_type<L: ArrowPrimitiveType, R: ArrowPrimitiveType>(
build_side: JoinSide,
prune_length: usize,
visited_rows: &HashSet<usize>,
deleted_offset: usize,
join_type: JoinType,
) -> Result<(PrimitiveArray<L>, PrimitiveArray<R>)>
where
NativeAdapter<L>: From<<L as ArrowPrimitiveType>::Native>,
{
// Store the result in a tuple
let result = match (build_side, join_type) {
// In the case of `Left` or `Right` join, or `Full` join, get the anti indices
(JoinSide::Left, JoinType::Left | JoinType::LeftAnti)
| (JoinSide::Right, JoinType::Right | JoinType::RightAnti)
| (_, JoinType::Full) => {
let build_unmatched_indices =
get_anti_indices(prune_length, deleted_offset, visited_rows);
let mut builder =
PrimitiveBuilder::<R>::with_capacity(build_unmatched_indices.len());
builder.append_nulls(build_unmatched_indices.len());
let probe_indices = builder.finish();
(build_unmatched_indices, probe_indices)
}
// In the case of `LeftSemi` or `RightSemi` join, get the semi indices
(JoinSide::Left, JoinType::LeftSemi) | (JoinSide::Right, JoinType::RightSemi) => {
let build_unmatched_indices =
get_semi_indices(prune_length, deleted_offset, visited_rows);
let mut builder =
PrimitiveBuilder::<R>::with_capacity(build_unmatched_indices.len());
builder.append_nulls(build_unmatched_indices.len());
let probe_indices = builder.finish();
(build_unmatched_indices, probe_indices)
}
// The case of other join types is not considered
_ => unreachable!(),
};
Ok(result)
}
struct OneSideHashJoiner {
/// Build side
build_side: JoinSide,
/// Input record batch buffer
input_buffer: RecordBatch,
/// Columns from the side
on: Vec<Column>,
/// Hashmap
hashmap: SymmetricJoinHashMap,
/// To optimize hash deleting in case of pruning, we hold them in memory
row_hash_values: VecDeque<u64>,
/// Reuse the hashes buffer
hashes_buffer: Vec<u64>,
/// Matched rows
visited_rows: HashSet<usize>,
/// Offset
offset: usize,
/// Deleted offset
deleted_offset: usize,
}
impl OneSideHashJoiner {
pub fn size(&self) -> usize {
let mut size = 0;
size += std::mem::size_of_val(self);
size += std::mem::size_of_val(&self.build_side);
size += self.input_buffer.get_array_memory_size();
size += std::mem::size_of_val(&self.on);
size += self.hashmap.size();
size += self.row_hash_values.capacity() * std::mem::size_of::<u64>();
size += self.hashes_buffer.capacity() * std::mem::size_of::<u64>();
size += self.visited_rows.capacity() * std::mem::size_of::<usize>();
size += std::mem::size_of_val(&self.offset);
size += std::mem::size_of_val(&self.deleted_offset);
size
}
pub fn new(build_side: JoinSide, on: Vec<Column>, schema: SchemaRef) -> Self {
Self {
build_side,
input_buffer: RecordBatch::new_empty(schema),
on,
hashmap: SymmetricJoinHashMap::with_capacity(0),
row_hash_values: VecDeque::new(),
hashes_buffer: vec![],
visited_rows: HashSet::new(),
offset: 0,
deleted_offset: 0,
}
}
pub fn update_hash(
on: &[Column],
batch: &RecordBatch,
hash_map: &mut SymmetricJoinHashMap,
offset: usize,
random_state: &RandomState,
hashes_buffer: &mut Vec<u64>,
) -> Result<()> {
// evaluate the keys
let keys_values = on
.iter()
.map(|c| Ok(c.evaluate(batch)?.into_array(batch.num_rows())))
.collect::<Result<Vec<_>>>()?;
// calculate the hash values
let hash_values = create_hashes(&keys_values, random_state, hashes_buffer)?;
// insert hashes to key of the hashmap
for (row, hash_value) in hash_values.iter().enumerate() {
let item = hash_map
.0
.get_mut(*hash_value, |(hash, _)| *hash_value == *hash);
if let Some((_, indices)) = item {
indices.push((row + offset) as u64);
} else {
hash_map.0.insert(
*hash_value,
(*hash_value, smallvec![(row + offset) as u64]),
|(hash, _)| *hash,
);
}
}
Ok(())
}
/// Updates the internal state of the [OneSideHashJoiner] with the incoming batch.
///
/// # Arguments
///
/// * `batch` - The incoming [RecordBatch] to be merged with the internal input buffer
/// * `random_state` - The random state used to hash values
///
/// # Returns
///
/// Returns a [Result] encapsulating any intermediate errors.
fn update_internal_state(
&mut self,
batch: &RecordBatch,
random_state: &RandomState,
) -> Result<()> {
// Merge the incoming batch with the existing input buffer:
self.input_buffer = concat_batches(&batch.schema(), [&self.input_buffer, batch])?;
// Resize the hashes buffer to the number of rows in the incoming batch:
self.hashes_buffer.resize(batch.num_rows(), 0);
// Get allocation_info before adding the item
// Update the hashmap with the join key values and hashes of the incoming batch:
Self::update_hash(
&self.on,
batch,
&mut self.hashmap,
self.offset,
random_state,
&mut self.hashes_buffer,
)?;
// Add the hashes buffer to the hash value deque:
self.row_hash_values.extend(self.hashes_buffer.iter());
Ok(())
}
/// Gets build and probe indices which satisfy the on condition (including
/// the equality condition and the join filter) in the join.
#[allow(clippy::too_many_arguments)]
pub fn build_join_indices(
probe_batch: &RecordBatch,
build_hashmap: &SymmetricJoinHashMap,
build_input_buffer: &RecordBatch,
on_build: &[Column],
on_probe: &[Column],
filter: Option<&JoinFilter>,
random_state: &RandomState,
null_equals_null: bool,
hashes_buffer: &mut Vec<u64>,
offset: Option<usize>,
build_side: JoinSide,
) -> Result<(UInt64Array, UInt32Array)> {
// Get the indices that satisfy the equality condition, like `left.a1 = right.a2`
let (build_indices, probe_indices) = Self::build_equal_condition_join_indices(
build_hashmap,
build_input_buffer,
probe_batch,
on_build,
on_probe,
random_state,
null_equals_null,
hashes_buffer,
offset,
)?;
if let Some(filter) = filter {
// Filter the indices which satisfy the non-equal join condition, like `left.b1 = 10`
apply_join_filter_to_indices(
build_input_buffer,
probe_batch,
build_indices,
probe_indices,
filter,
build_side,
)
} else {
Ok((build_indices, probe_indices))
}
}
// Returns build/probe indices satisfying the equality condition.
// On LEFT.b1 = RIGHT.b2
// LEFT Table:
// a1 b1 c1
// 1 1 10
// 3 3 30
// 5 5 50
// 7 7 70
// 9 8 90
// 11 8 110
// 13 10 130
// RIGHT Table:
// a2 b2 c2
// 2 2 20
// 4 4 40
// 6 6 60
// 8 8 80
// 10 10 100
// 12 10 120
// The result is
// "+----+----+-----+----+----+-----+",
// "| a1 | b1 | c1 | a2 | b2 | c2 |",
// "+----+----+-----+----+----+-----+",
// "| 11 | 8 | 110 | 8 | 8 | 80 |",
// "| 13 | 10 | 130 | 10 | 10 | 100 |",
// "| 13 | 10 | 130 | 12 | 10 | 120 |",
// "| 9 | 8 | 90 | 8 | 8 | 80 |",
// "+----+----+-----+----+----+-----+"
// And the result of build and probe indices are:
// Build indices: 5, 6, 6, 4
// Probe indices: 3, 4, 5, 3
#[allow(clippy::too_many_arguments)]
pub fn build_equal_condition_join_indices(
build_hashmap: &SymmetricJoinHashMap,
build_input_buffer: &RecordBatch,
probe_batch: &RecordBatch,
build_on: &[Column],
probe_on: &[Column],
random_state: &RandomState,
null_equals_null: bool,
hashes_buffer: &mut Vec<u64>,
offset: Option<usize>,
) -> Result<(UInt64Array, UInt32Array)> {
let keys_values = probe_on
.iter()
.map(|c| Ok(c.evaluate(probe_batch)?.into_array(probe_batch.num_rows())))
.collect::<Result<Vec<_>>>()?;
let build_join_values = build_on
.iter()
.map(|c| {
Ok(c.evaluate(build_input_buffer)?
.into_array(build_input_buffer.num_rows()))
})
.collect::<Result<Vec<_>>>()?;
hashes_buffer.clear();
hashes_buffer.resize(probe_batch.num_rows(), 0);
let hash_values = create_hashes(&keys_values, random_state, hashes_buffer)?;
// Using a buffer builder to avoid slower normal builder
let mut build_indices = UInt64BufferBuilder::new(0);
let mut probe_indices = UInt32BufferBuilder::new(0);
let offset_value = offset.unwrap_or(0);
// Visit all of the probe rows
for (row, hash_value) in hash_values.iter().enumerate() {
// Get the hash and find it in the build index
// For every item on the build and probe we check if it matches
// This possibly contains rows with hash collisions,
// So we have to check here whether rows are equal or not
if let Some((_, indices)) = build_hashmap
.0
.get(*hash_value, |(hash, _)| *hash_value == *hash)
{
for &i in indices {
// Check hash collisions
let offset_build_index = i as usize - offset_value;
// Check hash collisions
if equal_rows(
offset_build_index,
row,
&build_join_values,
&keys_values,
null_equals_null,
)? {
build_indices.append(offset_build_index as u64);
probe_indices.append(row as u32);
}
}
}
}
Ok((
PrimitiveArray::new(build_indices.finish().into(), None),
PrimitiveArray::new(probe_indices.finish().into(), None),
))
}
/// This method performs a join between the build side input buffer and the probe side batch.
///
/// # Arguments
///
/// * `schema` - A reference to the schema of the output record batch.
/// * `join_type` - The type of join to be performed.
/// * `on_probe` - An array of columns on which the join will be performed. The columns are from the probe side of the join.
/// * `filter` - An optional filter on the join condition.
/// * `probe_batch` - The second record batch to be joined.
/// * `probe_visited` - A hash set to store the visited indices from the probe batch.
/// * `probe_offset` - The offset of the probe side for visited indices calculations.
/// * `column_indices` - An array of columns to be selected for the result of the join.
/// * `random_state` - The random state for the join.
/// * `null_equals_null` - A boolean indicating whether NULL values should be treated as equal when joining.
///
/// # Returns
///
/// A [Result] containing an optional record batch if the join type is not one of `LeftAnti`, `RightAnti`, `LeftSemi` or `RightSemi`.
/// If the join type is one of the above four, the function will return [None].
#[allow(clippy::too_many_arguments)]
fn join_with_probe_batch(
&mut self,
schema: &SchemaRef,
join_type: JoinType,
on_probe: &[Column],
filter: Option<&JoinFilter>,
probe_batch: &RecordBatch,
probe_visited: &mut HashSet<usize>,
probe_offset: usize,
column_indices: &[ColumnIndex],
random_state: &RandomState,
null_equals_null: bool,
) -> Result<Option<RecordBatch>> {
if self.input_buffer.num_rows() == 0 || probe_batch.num_rows() == 0 {
return Ok(None);
}
let (build_indices, probe_indices) = Self::build_join_indices(
probe_batch,
&self.hashmap,
&self.input_buffer,
&self.on,
on_probe,
filter,
random_state,
null_equals_null,
&mut self.hashes_buffer,
Some(self.deleted_offset),
self.build_side,
)?;
if need_to_produce_result_in_final(self.build_side, join_type) {
record_visited_indices(
&mut self.visited_rows,
self.deleted_offset,
&build_indices,
);
}
if need_to_produce_result_in_final(self.build_side.negate(), join_type) {
record_visited_indices(probe_visited, probe_offset, &probe_indices);
}
if matches!(
join_type,
JoinType::LeftAnti
| JoinType::RightAnti
| JoinType::LeftSemi
| JoinType::RightSemi
) {
Ok(None)
} else {
build_batch_from_indices(
schema,
&self.input_buffer,
probe_batch,
&build_indices,
&probe_indices,
column_indices,
self.build_side,
)
.map(|batch| (batch.num_rows() > 0).then_some(batch))
}
}
/// This function produces unmatched record results based on the build side,
/// join type and other parameters.
///
/// The method uses first `prune_length` rows from the build side input buffer
/// to produce results.
///
/// # Arguments
///
/// * `output_schema` - The schema of the final output record batch.
/// * `prune_length` - The length of the determined prune length.
/// * `probe_schema` - The schema of the probe [RecordBatch].
/// * `join_type` - The type of join to be performed.
/// * `column_indices` - Indices of columns that are being joined.
///
/// # Returns
///
/// * `Option<RecordBatch>` - The final output record batch if required, otherwise [None].
fn build_side_determined_results(
&self,
output_schema: &SchemaRef,
prune_length: usize,
probe_schema: SchemaRef,
join_type: JoinType,
column_indices: &[ColumnIndex],
) -> Result<Option<RecordBatch>> {
// Check if we need to produce a result in the final output:
if need_to_produce_result_in_final(self.build_side, join_type) {
// Calculate the indices for build and probe sides based on join type and build side:
let (build_indices, probe_indices) = calculate_indices_by_join_type(
self.build_side,
prune_length,
&self.visited_rows,
self.deleted_offset,
join_type,
)?;
// Create an empty probe record batch:
let empty_probe_batch = RecordBatch::new_empty(probe_schema);
// Build the final result from the indices of build and probe sides:
build_batch_from_indices(
output_schema.as_ref(),
&self.input_buffer,
&empty_probe_batch,
&build_indices,
&probe_indices,
column_indices,
self.build_side,
)
.map(|batch| (batch.num_rows() > 0).then_some(batch))
} else {
// If we don't need to produce a result, return None
Ok(None)
}
}
/// Prunes the internal buffer.
///
/// Argument `probe_batch` is used to update the intervals of the sorted
/// filter expressions. The updated build interval determines the new length
/// of the build side. If there are rows to prune, they are removed from the
/// internal buffer.
///
/// # Arguments
///
/// * `schema` - The schema of the final output record batch
/// * `probe_batch` - Incoming RecordBatch of the probe side.
/// * `probe_side_sorted_filter_expr` - Probe side mutable sorted filter expression.
/// * `join_type` - The type of join (e.g. inner, left, right, etc.).
/// * `column_indices` - A vector of column indices that specifies which columns from the
/// build side should be included in the output.
/// * `graph` - A mutable reference to the physical expression graph.
///
/// # Returns
///
/// If there are rows to prune, returns the pruned build side record batch wrapped in an `Ok` variant.
/// Otherwise, returns `Ok(None)`.
fn calculate_prune_length_with_probe_batch(
&mut self,
build_side_sorted_filter_expr: &mut SortedFilterExpr,
probe_side_sorted_filter_expr: &mut SortedFilterExpr,
graph: &mut ExprIntervalGraph,
) -> Result<usize> {
// Return early if the input buffer is empty:
if self.input_buffer.num_rows() == 0 {
return Ok(0);
}
// Process the build and probe side sorted filter expressions if both are present:
// Collect the sorted filter expressions into a vector of (node_index, interval) tuples:
let mut filter_intervals = vec![];
for expr in [
&build_side_sorted_filter_expr,
&probe_side_sorted_filter_expr,
] {
filter_intervals.push((expr.node_index(), expr.interval().clone()))
}
// Update the physical expression graph using the join filter intervals:
graph.update_ranges(&mut filter_intervals)?;
// Extract the new join filter interval for the build side:
let calculated_build_side_interval = filter_intervals.remove(0).1;
// If the intervals have not changed, return early without pruning:
if calculated_build_side_interval.eq(build_side_sorted_filter_expr.interval()) {
return Ok(0);
}
// Update the build side interval and determine the pruning length:
build_side_sorted_filter_expr.set_interval(calculated_build_side_interval);
determine_prune_length(&self.input_buffer, build_side_sorted_filter_expr)
}
fn prune_internal_state_and_build_anti_result(
&mut self,
prune_length: usize,
schema: &SchemaRef,
probe_batch: &RecordBatch,
join_type: JoinType,
column_indices: &[ColumnIndex],
) -> Result<Option<RecordBatch>> {
// Compute the result and perform pruning if there are rows to prune:
let result = self.build_side_determined_results(
schema,
prune_length,
probe_batch.schema(),
join_type,
column_indices,
);
// Prune the hash values:
prune_hash_values(
prune_length,
&mut self.hashmap,
&mut self.row_hash_values,
self.deleted_offset as u64,
)?;
// Remove pruned rows from the visited rows set:
for row in self.deleted_offset..(self.deleted_offset + prune_length) {
self.visited_rows.remove(&row);
}
// Update the input buffer after pruning:
self.input_buffer = self
.input_buffer
.slice(prune_length, self.input_buffer.num_rows() - prune_length);
// Increment the deleted offset:
self.deleted_offset += prune_length;
result
}
}
fn combine_two_batches(
output_schema: &SchemaRef,
left_batch: Option<RecordBatch>,
right_batch: Option<RecordBatch>,
) -> Result<Option<RecordBatch>> {
match (left_batch, right_batch) {
(Some(batch), None) | (None, Some(batch)) => {
// If only one of the batches are present, return it:
Ok(Some(batch))
}
(Some(left_batch), Some(right_batch)) => {
// If both batches are present, concatenate them:
concat_batches(output_schema, &[left_batch, right_batch])
.map_err(DataFusionError::ArrowError)
.map(Some)
}
(None, None) => {
// If neither is present, return an empty batch:
Ok(None)
}
}
}
impl SymmetricHashJoinStream {
fn size(&self) -> usize {
let mut size = 0;
size += std::mem::size_of_val(&self.input_stream);
size += std::mem::size_of_val(&self.schema);
size += std::mem::size_of_val(&self.filter);
size += std::mem::size_of_val(&self.join_type);
size += self.left.size();
size += self.right.size();
size += std::mem::size_of_val(&self.column_indices);
size += self.graph.as_ref().map(|g| g.size()).unwrap_or(0);
size += std::mem::size_of_val(&self.left_sorted_filter_expr);
size += std::mem::size_of_val(&self.right_sorted_filter_expr);
size += std::mem::size_of_val(&self.random_state);
size += std::mem::size_of_val(&self.null_equals_null);
size += std::mem::size_of_val(&self.metrics);
size += std::mem::size_of_val(&self.final_result);
size
}
/// Polls the next result of the join operation.
///
/// If the result of the join is ready, it returns the next record batch.
/// If the join has completed and there are no more results, it returns
/// `Poll::Ready(None)`. If the join operation is not complete, but the
/// current stream is not ready yet, it returns `Poll::Pending`.
fn poll_next_impl(
&mut self,
cx: &mut std::task::Context<'_>,
) -> Poll<Option<Result<RecordBatch>>> {
loop {
// Poll the next batch from `input_stream`:
match self.input_stream.poll_next_unpin(cx) {
// Batch is available
Poll::Ready(Some((side, Ok(probe_batch)))) => {
// Determine which stream should be polled next. The side the
// RecordBatch comes from becomes the probe side.
let (
probe_hash_joiner,
build_hash_joiner,
probe_side_sorted_filter_expr,
build_side_sorted_filter_expr,
probe_side_metrics,
) = if side.eq(&JoinSide::Left) {
(
&mut self.left,
&mut self.right,
&mut self.left_sorted_filter_expr,
&mut self.right_sorted_filter_expr,
&mut self.metrics.left,
)
} else {
(
&mut self.right,
&mut self.left,
&mut self.right_sorted_filter_expr,
&mut self.left_sorted_filter_expr,
&mut self.metrics.right,
)
};
// Update the metrics for the stream that was polled:
probe_side_metrics.input_batches.add(1);
probe_side_metrics.input_rows.add(probe_batch.num_rows());
// Update the internal state of the hash joiner for the build side:
probe_hash_joiner
.update_internal_state(&probe_batch, &self.random_state)?;
// Join the two sides:
let equal_result = build_hash_joiner.join_with_probe_batch(
&self.schema,
self.join_type,
&probe_hash_joiner.on,
self.filter.as_ref(),
&probe_batch,
&mut probe_hash_joiner.visited_rows,
probe_hash_joiner.offset,
&self.column_indices,
&self.random_state,
self.null_equals_null,
)?;
// Increment the offset for the probe hash joiner:
probe_hash_joiner.offset += probe_batch.num_rows();
let anti_result = if let (
Some(build_side_sorted_filter_expr),
Some(probe_side_sorted_filter_expr),
Some(graph),
) = (
build_side_sorted_filter_expr.as_mut(),
probe_side_sorted_filter_expr.as_mut(),
self.graph.as_mut(),
) {
// Calculate filter intervals:
calculate_filter_expr_intervals(
&build_hash_joiner.input_buffer,
build_side_sorted_filter_expr,
&probe_batch,
probe_side_sorted_filter_expr,
)?;
let prune_length = build_hash_joiner
.calculate_prune_length_with_probe_batch(
build_side_sorted_filter_expr,
probe_side_sorted_filter_expr,
graph,
)?;
if prune_length > 0 {
build_hash_joiner.prune_internal_state_and_build_anti_result(
prune_length,
&self.schema,
&probe_batch,
self.join_type,
&self.column_indices,
)?
} else {
None
}
} else {
None
};
// Combine results:
let result =
combine_two_batches(&self.schema, equal_result, anti_result)?;
let capacity = self.size();
self.metrics.stream_memory_usage.set(capacity);
self.reservation.lock().try_resize(capacity)?;
// Update the metrics if we have a batch; otherwise, continue the loop.
if let Some(batch) = &result {
self.metrics.output_batches.add(1);
self.metrics.output_rows.add(batch.num_rows());
return Poll::Ready(Ok(result).transpose());
}
}
Poll::Ready(Some((_, Err(e)))) => return Poll::Ready(Some(Err(e))),
Poll::Ready(None) => {
// If the final result has already been obtained, return `Poll::Ready(None)`:
if self.final_result {
return Poll::Ready(None);
}
self.final_result = true;
// Get the left side results:
let left_result = self.left.build_side_determined_results(
&self.schema,
self.left.input_buffer.num_rows(),
self.right.input_buffer.schema(),
self.join_type,
&self.column_indices,
)?;
// Get the right side results:
let right_result = self.right.build_side_determined_results(
&self.schema,
self.right.input_buffer.num_rows(),
self.left.input_buffer.schema(),
self.join_type,
&self.column_indices,
)?;
// Combine the left and right results:
let result =
combine_two_batches(&self.schema, left_result, right_result)?;
// Update the metrics and return the result:
if let Some(batch) = &result {
// Update the metrics:
self.metrics.output_batches.add(1);
self.metrics.output_rows.add(batch.num_rows());
return Poll::Ready(Ok(result).transpose());
}
}
Poll::Pending => return Poll::Pending,
}
}
}
}
#[cfg(test)]
mod tests {
use std::fs::File;
use arrow::array::{ArrayRef, Float64Array, IntervalDayTimeArray};
use arrow::array::{Int32Array, TimestampMillisecondArray};
use arrow::compute::SortOptions;
use arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit};
use arrow::util::pretty::pretty_format_batches;
use rstest::*;
use tempfile::TempDir;
use datafusion_expr::Operator;
use datafusion_physical_expr::expressions::{binary, col, Column};
use datafusion_physical_expr::intervals::test_utils::{
gen_conjunctive_numerical_expr, gen_conjunctive_temporal_expr,
};
use datafusion_physical_expr::PhysicalExpr;
use crate::physical_plan::joins::{
hash_join_utils::tests::complicated_filter, HashJoinExec, PartitionMode,
};
use crate::physical_plan::{
common, displayable, memory::MemoryExec, repartition::RepartitionExec,
};
use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext};
use crate::test_util::register_unbounded_file_with_ordering;
use super::*;
const TABLE_SIZE: i32 = 100;
fn compare_batches(collected_1: &[RecordBatch], collected_2: &[RecordBatch]) {
// compare
let first_formatted = pretty_format_batches(collected_1).unwrap().to_string();
let second_formatted = pretty_format_batches(collected_2).unwrap().to_string();
let mut first_formatted_sorted: Vec<&str> =
first_formatted.trim().lines().collect();
first_formatted_sorted.sort_unstable();
let mut second_formatted_sorted: Vec<&str> =
second_formatted.trim().lines().collect();
second_formatted_sorted.sort_unstable();
for (i, (first_line, second_line)) in first_formatted_sorted
.iter()
.zip(&second_formatted_sorted)
.enumerate()
{
assert_eq!((i, first_line), (i, second_line));
}
}
async fn partitioned_sym_join_with_filter(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
filter: Option<JoinFilter>,
join_type: &JoinType,
null_equals_null: bool,
context: Arc<TaskContext>,
) -> Result<Vec<RecordBatch>> {
let partition_count = 4;
let left_expr = on
.iter()
.map(|(l, _)| Arc::new(l.clone()) as _)
.collect::<Vec<_>>();
let right_expr = on
.iter()
.map(|(_, r)| Arc::new(r.clone()) as _)
.collect::<Vec<_>>();
let join = SymmetricHashJoinExec::try_new(
Arc::new(RepartitionExec::try_new(
left,
Partitioning::Hash(left_expr, partition_count),
)?),
Arc::new(RepartitionExec::try_new(
right,
Partitioning::Hash(right_expr, partition_count),
)?),
on,
filter,
join_type,
null_equals_null,
)?;
let mut batches = vec![];
for i in 0..partition_count {
let stream = join.execute(i, context.clone())?;
let more_batches = common::collect(stream).await?;
batches.extend(
more_batches
.into_iter()
.filter(|b| b.num_rows() > 0)
.collect::<Vec<_>>(),
);
}
Ok(batches)
}
async fn partitioned_hash_join_with_filter(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
on: JoinOn,
filter: Option<JoinFilter>,
join_type: &JoinType,
null_equals_null: bool,
context: Arc<TaskContext>,
) -> Result<Vec<RecordBatch>> {
let partition_count = 4;
let (left_expr, right_expr) = on
.iter()
.map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _))
.unzip();
let join = HashJoinExec::try_new(
Arc::new(RepartitionExec::try_new(
left,
Partitioning::Hash(left_expr, partition_count),
)?),
Arc::new(RepartitionExec::try_new(
right,
Partitioning::Hash(right_expr, partition_count),
)?),
on,
filter,
join_type,
PartitionMode::Partitioned,
null_equals_null,
)?;
let mut batches = vec![];
for i in 0..partition_count {
let stream = join.execute(i, context.clone())?;
let more_batches = common::collect(stream).await?;
batches.extend(
more_batches
.into_iter()
.filter(|b| b.num_rows() > 0)
.collect::<Vec<_>>(),
);
}
Ok(batches)
}
pub fn split_record_batches(
batch: &RecordBatch,
batch_size: usize,
) -> Result<Vec<RecordBatch>> {
let row_num = batch.num_rows();
let number_of_batch = row_num / batch_size;
let mut sizes = vec![batch_size; number_of_batch];
sizes.push(row_num - (batch_size * number_of_batch));
let mut result = vec![];
for (i, size) in sizes.iter().enumerate() {
result.push(batch.slice(i * batch_size, *size));
}
Ok(result)
}
// It creates join filters for different type of fields for testing.
macro_rules! join_expr_tests {
($func_name:ident, $type:ty, $SCALAR:ident) => {
fn $func_name(
expr_id: usize,
left_col: Arc<dyn PhysicalExpr>,
right_col: Arc<dyn PhysicalExpr>,
) -> Arc<dyn PhysicalExpr> {
match expr_id {
// left_col + 1 > right_col + 5 AND left_col + 3 < right_col + 10
0 => gen_conjunctive_numerical_expr(
left_col,
right_col,
(
Operator::Plus,
Operator::Plus,
Operator::Plus,
Operator::Plus,
),
ScalarValue::$SCALAR(Some(1 as $type)),
ScalarValue::$SCALAR(Some(5 as $type)),
ScalarValue::$SCALAR(Some(3 as $type)),
ScalarValue::$SCALAR(Some(10 as $type)),
(Operator::Gt, Operator::Lt),
),
// left_col - 1 > right_col + 5 AND left_col + 3 < right_col + 10
1 => gen_conjunctive_numerical_expr(
left_col,
right_col,
(
Operator::Minus,
Operator::Plus,
Operator::Plus,
Operator::Plus,
),
ScalarValue::$SCALAR(Some(1 as $type)),
ScalarValue::$SCALAR(Some(5 as $type)),
ScalarValue::$SCALAR(Some(3 as $type)),
ScalarValue::$SCALAR(Some(10 as $type)),
(Operator::Gt, Operator::Lt),
),
// left_col - 1 > right_col + 5 AND left_col - 3 < right_col + 10
2 => gen_conjunctive_numerical_expr(
left_col,
right_col,
(
Operator::Minus,
Operator::Plus,
Operator::Minus,
Operator::Plus,
),
ScalarValue::$SCALAR(Some(1 as $type)),
ScalarValue::$SCALAR(Some(5 as $type)),
ScalarValue::$SCALAR(Some(3 as $type)),
ScalarValue::$SCALAR(Some(10 as $type)),
(Operator::Gt, Operator::Lt),
),
// left_col - 10 > right_col - 5 AND left_col - 3 < right_col + 10
3 => gen_conjunctive_numerical_expr(
left_col,
right_col,
(
Operator::Minus,
Operator::Minus,
Operator::Minus,
Operator::Plus,
),
ScalarValue::$SCALAR(Some(10 as $type)),
ScalarValue::$SCALAR(Some(5 as $type)),
ScalarValue::$SCALAR(Some(3 as $type)),
ScalarValue::$SCALAR(Some(10 as $type)),
(Operator::Gt, Operator::Lt),
),
// left_col - 10 > right_col - 5 AND left_col - 30 < right_col - 3
4 => gen_conjunctive_numerical_expr(
left_col,
right_col,
(
Operator::Minus,
Operator::Minus,
Operator::Minus,
Operator::Minus,
),
ScalarValue::$SCALAR(Some(10 as $type)),
ScalarValue::$SCALAR(Some(5 as $type)),
ScalarValue::$SCALAR(Some(30 as $type)),
ScalarValue::$SCALAR(Some(3 as $type)),
(Operator::Gt, Operator::Lt),
),
// left_col - 2 >= right_col - 5 AND left_col - 7 <= right_col - 3
5 => gen_conjunctive_numerical_expr(
left_col,
right_col,
(
Operator::Minus,
Operator::Plus,
Operator::Plus,
Operator::Minus,
),
ScalarValue::$SCALAR(Some(2 as $type)),
ScalarValue::$SCALAR(Some(5 as $type)),
ScalarValue::$SCALAR(Some(7 as $type)),
ScalarValue::$SCALAR(Some(3 as $type)),
(Operator::GtEq, Operator::LtEq),
),
// left_col - 28 >= right_col - 11 AND left_col - 21 <= right_col - 39
6 => gen_conjunctive_numerical_expr(
left_col,
right_col,
(
Operator::Plus,
Operator::Minus,
Operator::Plus,
Operator::Plus,
),
ScalarValue::$SCALAR(Some(28 as $type)),
ScalarValue::$SCALAR(Some(11 as $type)),
ScalarValue::$SCALAR(Some(21 as $type)),
ScalarValue::$SCALAR(Some(39 as $type)),
(Operator::Gt, Operator::LtEq),
),
// left_col - 28 >= right_col - 11 AND left_col - 21 <= right_col - 39
7 => gen_conjunctive_numerical_expr(
left_col,
right_col,
(
Operator::Plus,
Operator::Minus,
Operator::Minus,
Operator::Plus,
),
ScalarValue::$SCALAR(Some(28 as $type)),
ScalarValue::$SCALAR(Some(11 as $type)),
ScalarValue::$SCALAR(Some(21 as $type)),
ScalarValue::$SCALAR(Some(39 as $type)),
(Operator::GtEq, Operator::Lt),
),
_ => panic!("No case"),
}
}
};
}
join_expr_tests!(join_expr_tests_fixture_i32, i32, Int32);
join_expr_tests!(join_expr_tests_fixture_f64, f64, Float64);
use rand::rngs::StdRng;
use rand::{Rng, SeedableRng};
use std::iter::Iterator;
struct AscendingRandomFloatIterator {
prev: f64,
max: f64,
rng: StdRng,
}
impl AscendingRandomFloatIterator {
fn new(min: f64, max: f64) -> Self {
let mut rng = StdRng::seed_from_u64(42);
let initial = rng.gen_range(min..max);
AscendingRandomFloatIterator {
prev: initial,
max,
rng,
}
}
}
impl Iterator for AscendingRandomFloatIterator {
type Item = f64;
fn next(&mut self) -> Option<Self::Item> {
let value = self.rng.gen_range(self.prev..self.max);
self.prev = value;
Some(value)
}
}
fn join_expr_tests_fixture_temporal(
expr_id: usize,
left_col: Arc<dyn PhysicalExpr>,
right_col: Arc<dyn PhysicalExpr>,
schema: &Schema,
) -> Result<Arc<dyn PhysicalExpr>> {
match expr_id {
// constructs ((left_col - INTERVAL '100ms') > (right_col - INTERVAL '200ms')) AND ((left_col - INTERVAL '450ms') < (right_col - INTERVAL '300ms'))
0 => gen_conjunctive_temporal_expr(
left_col,
right_col,
Operator::Minus,
Operator::Minus,
Operator::Minus,
Operator::Minus,
ScalarValue::new_interval_dt(0, 100), // 100 ms
ScalarValue::new_interval_dt(0, 200), // 200 ms
ScalarValue::new_interval_dt(0, 450), // 450 ms
ScalarValue::new_interval_dt(0, 300), // 300 ms
schema,
),
// constructs ((left_col - TIMESTAMP '2023-01-01:12.00.03') > (right_col - TIMESTAMP '2023-01-01:12.00.01')) AND ((left_col - TIMESTAMP '2023-01-01:12.00.00') < (right_col - TIMESTAMP '2023-01-01:12.00.02'))
1 => gen_conjunctive_temporal_expr(
left_col,
right_col,
Operator::Minus,
Operator::Minus,
Operator::Minus,
Operator::Minus,
ScalarValue::TimestampMillisecond(Some(1672574403000), None), // 2023-01-01:12.00.03
ScalarValue::TimestampMillisecond(Some(1672574401000), None), // 2023-01-01:12.00.01
ScalarValue::TimestampMillisecond(Some(1672574400000), None), // 2023-01-01:12.00.00
ScalarValue::TimestampMillisecond(Some(1672574402000), None), // 2023-01-01:12.00.02
schema,
),
_ => unreachable!(),
}
}
fn build_sides_record_batches(
table_size: i32,
key_cardinality: (i32, i32),
) -> Result<(RecordBatch, RecordBatch)> {
let null_ratio: f64 = 0.4;
let initial_range = 0..table_size;
let index = (table_size as f64 * null_ratio).round() as i32;
let rest_of = index..table_size;
let ordered: ArrayRef = Arc::new(Int32Array::from_iter(
initial_range.clone().collect::<Vec<i32>>(),
));
let ordered_des = Arc::new(Int32Array::from_iter(
initial_range.clone().rev().collect::<Vec<i32>>(),
));
let cardinality = Arc::new(Int32Array::from_iter(
initial_range.clone().map(|x| x % 4).collect::<Vec<i32>>(),
));
let cardinality_key_left = Arc::new(Int32Array::from_iter(
initial_range
.clone()
.map(|x| x % key_cardinality.0)
.collect::<Vec<i32>>(),
));
let cardinality_key_right = Arc::new(Int32Array::from_iter(
initial_range
.clone()
.map(|x| x % key_cardinality.1)
.collect::<Vec<i32>>(),
));
let ordered_asc_null_first = Arc::new(Int32Array::from_iter({
std::iter::repeat(None)
.take(index as usize)
.chain(rest_of.clone().map(Some))
.collect::<Vec<Option<i32>>>()
}));
let ordered_asc_null_last = Arc::new(Int32Array::from_iter({
rest_of
.clone()
.map(Some)
.chain(std::iter::repeat(None).take(index as usize))
.collect::<Vec<Option<i32>>>()
}));
let ordered_desc_null_first = Arc::new(Int32Array::from_iter({
std::iter::repeat(None)
.take(index as usize)
.chain(rest_of.rev().map(Some))
.collect::<Vec<Option<i32>>>()
}));
let time = Arc::new(TimestampMillisecondArray::from(
initial_range
.clone()
.map(|x| x as i64 + 1672531200000) // x + 2023-01-01:00.00.00
.collect::<Vec<i64>>(),
));
let interval_time: ArrayRef = Arc::new(IntervalDayTimeArray::from(
initial_range
.map(|x| x as i64 * 100) // x * 100ms
.collect::<Vec<i64>>(),
));
let float_asc = Arc::new(Float64Array::from_iter_values(
AscendingRandomFloatIterator::new(0., table_size as f64)
.take(table_size as usize),
));
let left = RecordBatch::try_from_iter(vec![
("la1", ordered.clone()),
("lb1", cardinality.clone()),
("lc1", cardinality_key_left),
("lt1", time.clone()),
("la2", ordered.clone()),
("la1_des", ordered_des.clone()),
("l_asc_null_first", ordered_asc_null_first.clone()),
("l_asc_null_last", ordered_asc_null_last.clone()),
("l_desc_null_first", ordered_desc_null_first.clone()),
("li1", interval_time.clone()),
("l_float", float_asc.clone()),
])?;
let right = RecordBatch::try_from_iter(vec![
("ra1", ordered.clone()),
("rb1", cardinality),
("rc1", cardinality_key_right),
("rt1", time),
("ra2", ordered),
("ra1_des", ordered_des),
("r_asc_null_first", ordered_asc_null_first),
("r_asc_null_last", ordered_asc_null_last),
("r_desc_null_first", ordered_desc_null_first),
("ri1", interval_time),
("r_float", float_asc),
])?;
Ok((left, right))
}
fn create_memory_table(
left_batch: RecordBatch,
right_batch: RecordBatch,
left_sorted: Option<Vec<PhysicalSortExpr>>,
right_sorted: Option<Vec<PhysicalSortExpr>>,
batch_size: usize,
) -> Result<(Arc<dyn ExecutionPlan>, Arc<dyn ExecutionPlan>)> {
let mut left = MemoryExec::try_new(
&[split_record_batches(&left_batch, batch_size)?],
left_batch.schema(),
None,
)?;
if let Some(sorted) = left_sorted {
left = left.with_sort_information(sorted);
}
let mut right = MemoryExec::try_new(
&[split_record_batches(&right_batch, batch_size)?],
right_batch.schema(),
None,
)?;
if let Some(sorted) = right_sorted {
right = right.with_sort_information(sorted);
}
Ok((Arc::new(left), Arc::new(right)))
}
async fn experiment(
left: Arc<dyn ExecutionPlan>,
right: Arc<dyn ExecutionPlan>,
filter: Option<JoinFilter>,
join_type: JoinType,
on: JoinOn,
task_ctx: Arc<TaskContext>,
) -> Result<()> {
let first_batches = partitioned_sym_join_with_filter(
left.clone(),
right.clone(),
on.clone(),
filter.clone(),
&join_type,
false,
task_ctx.clone(),
)
.await?;
let second_batches = partitioned_hash_join_with_filter(
left, right, on, filter, &join_type, false, task_ctx,
)
.await?;
compare_batches(&first_batches, &second_batches);
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn complex_join_all_one_ascending_numeric(
#[values(
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::RightSemi,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::Full
)]
join_type: JoinType,
#[values(
(4, 5),
(11, 21),
(31, 71),
(99, 12),
)]
cardinality: (i32, i32),
) -> Result<()> {
// a + b > c + 10 AND a + b < c + 100
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let left_sorted = vec![PhysicalSortExpr {
expr: binary(
col("la1", left_schema)?,
Operator::Plus,
col("la2", left_schema)?,
left_schema,
)?,
options: SortOptions::default(),
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("ra1", right_schema)?,
options: SortOptions::default(),
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("0", DataType::Int32, true),
Field::new("1", DataType::Int32, true),
Field::new("2", DataType::Int32, true),
]);
let filter_expr = complicated_filter(&intermediate_schema)?;
let column_indices = vec![
ColumnIndex {
index: 0,
side: JoinSide::Left,
},
ColumnIndex {
index: 4,
side: JoinSide::Left,
},
ColumnIndex {
index: 0,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn join_all_one_ascending_numeric(
#[values(
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::RightSemi,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::Full
)]
join_type: JoinType,
#[values(
(4, 5),
(11, 21),
(31, 71),
(99, 12),
)]
cardinality: (i32, i32),
#[values(0, 1, 2, 3, 4, 5, 6, 7)] case_expr: usize,
) -> Result<()> {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let left_sorted = vec![PhysicalSortExpr {
expr: col("la1", left_schema)?,
options: SortOptions::default(),
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("ra1", right_schema)?,
options: SortOptions::default(),
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("left", DataType::Int32, true),
Field::new("right", DataType::Int32, true),
]);
let filter_expr = join_expr_tests_fixture_i32(
case_expr,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
);
let column_indices = vec![
ColumnIndex {
index: 0,
side: JoinSide::Left,
},
ColumnIndex {
index: 0,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn join_without_sort_information(
#[values(
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::RightSemi,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::Full
)]
join_type: JoinType,
#[values(
(4, 5),
(11, 21),
(31, 71),
(99, 12),
)]
cardinality: (i32, i32),
#[values(0, 1, 2, 3, 4, 5, 6)] case_expr: usize,
) -> Result<()> {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let (left, right) = create_memory_table(left_batch, right_batch, None, None, 13)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("left", DataType::Int32, true),
Field::new("right", DataType::Int32, true),
]);
let filter_expr = join_expr_tests_fixture_i32(
case_expr,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
);
let column_indices = vec![
ColumnIndex {
index: 5,
side: JoinSide::Left,
},
ColumnIndex {
index: 5,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn join_without_filter(
#[values(
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::RightSemi,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::Full
)]
join_type: JoinType,
) -> Result<()> {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) = build_sides_record_batches(TABLE_SIZE, (11, 21))?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let (left, right) = create_memory_table(left_batch, right_batch, None, None, 13)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
experiment(left, right, None, join_type, on, task_ctx).await?;
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn join_all_one_descending_numeric_particular(
#[values(
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::RightSemi,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::Full
)]
join_type: JoinType,
#[values(
(4, 5),
(11, 21),
(31, 71),
(99, 12),
)]
cardinality: (i32, i32),
#[values(0, 1, 2, 3, 4, 5, 6)] case_expr: usize,
) -> Result<()> {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let left_sorted = vec![PhysicalSortExpr {
expr: col("la1_des", left_schema)?,
options: SortOptions {
descending: true,
nulls_first: true,
},
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("ra1_des", right_schema)?,
options: SortOptions {
descending: true,
nulls_first: true,
},
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("left", DataType::Int32, true),
Field::new("right", DataType::Int32, true),
]);
let filter_expr = join_expr_tests_fixture_i32(
case_expr,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
);
let column_indices = vec![
ColumnIndex {
index: 5,
side: JoinSide::Left,
},
ColumnIndex {
index: 5,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[tokio::test]
async fn join_change_in_planner() -> Result<()> {
let config = SessionConfig::new().with_target_partitions(8);
let ctx = SessionContext::with_config(config);
let tmp_dir = TempDir::new().unwrap();
let left_file_path = tmp_dir.path().join("left.csv");
File::create(left_file_path.clone()).unwrap();
// Create schema
let schema = Arc::new(Schema::new(vec![
Field::new("a1", DataType::UInt32, false),
Field::new("a2", DataType::UInt32, false),
]));
// Specify the ordering:
let file_sort_order = vec![[datafusion_expr::col("a1")]
.into_iter()
.map(|e| {
let ascending = true;
let nulls_first = false;
e.sort(ascending, nulls_first)
})
.collect::<Vec<_>>()];
register_unbounded_file_with_ordering(
&ctx,
schema.clone(),
&left_file_path,
"left",
file_sort_order.clone(),
true,
)
.await?;
let right_file_path = tmp_dir.path().join("right.csv");
File::create(right_file_path.clone()).unwrap();
register_unbounded_file_with_ordering(
&ctx,
schema,
&right_file_path,
"right",
file_sort_order,
true,
)
.await?;
let sql = "SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10";
let dataframe = ctx.sql(sql).await?;
let physical_plan = dataframe.create_physical_plan().await?;
let formatted = displayable(physical_plan.as_ref()).indent(true).to_string();
let expected = {
[
"SymmetricHashJoinExec: join_type=Full, on=[(a2@1, a2@1)], filter=CAST(a1@0 AS Int64) > CAST(a1@1 AS Int64) + 3 AND CAST(a1@0 AS Int64) < CAST(a1@1 AS Int64) + 10",
" CoalesceBatchesExec: target_batch_size=8192",
" RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1",
// " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false",
" CoalesceBatchesExec: target_batch_size=8192",
" RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1",
// " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false"
]
};
let mut actual: Vec<&str> = formatted.trim().lines().collect();
// Remove CSV lines
actual.remove(3);
actual.remove(5);
assert_eq!(
expected,
actual[..],
"\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n"
);
Ok(())
}
#[tokio::test]
async fn join_change_in_planner_without_sort() -> Result<()> {
let config = SessionConfig::new().with_target_partitions(8);
let ctx = SessionContext::with_config(config);
let tmp_dir = TempDir::new()?;
let left_file_path = tmp_dir.path().join("left.csv");
File::create(left_file_path.clone())?;
let schema = Arc::new(Schema::new(vec![
Field::new("a1", DataType::UInt32, false),
Field::new("a2", DataType::UInt32, false),
]));
ctx.register_csv(
"left",
left_file_path.as_os_str().to_str().unwrap(),
CsvReadOptions::new().schema(&schema).mark_infinite(true),
)
.await?;
let right_file_path = tmp_dir.path().join("right.csv");
File::create(right_file_path.clone())?;
ctx.register_csv(
"right",
right_file_path.as_os_str().to_str().unwrap(),
CsvReadOptions::new().schema(&schema).mark_infinite(true),
)
.await?;
let sql = "SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10";
let dataframe = ctx.sql(sql).await?;
let physical_plan = dataframe.create_physical_plan().await?;
let formatted = displayable(physical_plan.as_ref()).indent(true).to_string();
let expected = {
[
"SymmetricHashJoinExec: join_type=Full, on=[(a2@1, a2@1)], filter=CAST(a1@0 AS Int64) > CAST(a1@1 AS Int64) + 3 AND CAST(a1@0 AS Int64) < CAST(a1@1 AS Int64) + 10",
" CoalesceBatchesExec: target_batch_size=8192",
" RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1",
// " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false",
" CoalesceBatchesExec: target_batch_size=8192",
" RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=1",
// " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false"
]
};
let mut actual: Vec<&str> = formatted.trim().lines().collect();
// Remove CSV lines
actual.remove(3);
actual.remove(5);
assert_eq!(
expected,
actual[..],
"\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n"
);
Ok(())
}
#[tokio::test]
async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> {
let config = SessionConfig::new()
.with_target_partitions(8)
.with_allow_symmetric_joins_without_pruning(false);
let ctx = SessionContext::with_config(config);
let tmp_dir = TempDir::new()?;
let left_file_path = tmp_dir.path().join("left.csv");
File::create(left_file_path.clone())?;
let schema = Arc::new(Schema::new(vec![
Field::new("a1", DataType::UInt32, false),
Field::new("a2", DataType::UInt32, false),
]));
ctx.register_csv(
"left",
left_file_path.as_os_str().to_str().unwrap(),
CsvReadOptions::new().schema(&schema).mark_infinite(true),
)
.await?;
let right_file_path = tmp_dir.path().join("right.csv");
File::create(right_file_path.clone())?;
ctx.register_csv(
"right",
right_file_path.as_os_str().to_str().unwrap(),
CsvReadOptions::new().schema(&schema).mark_infinite(true),
)
.await?;
let df = ctx.sql("SELECT t1.a1, t1.a2, t2.a1, t2.a2 FROM left as t1 FULL JOIN right as t2 ON t1.a2 = t2.a2 AND t1.a1 > t2.a1 + 3 AND t1.a1 < t2.a1 + 10").await?;
match df.create_physical_plan().await {
Ok(_) => panic!("Expecting error."),
Err(e) => {
assert_eq!(e.to_string(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on a non-prunable stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag")
}
}
Ok(())
}
#[tokio::test(flavor = "multi_thread")]
async fn build_null_columns_first() -> Result<()> {
let join_type = JoinType::Full;
let cardinality = (10, 11);
let case_expr = 1;
let config = SessionConfig::new().with_repartition_joins(false);
let session_ctx = SessionContext::with_config(config);
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let left_sorted = vec![PhysicalSortExpr {
expr: col("l_asc_null_first", left_schema)?,
options: SortOptions {
descending: false,
nulls_first: true,
},
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("r_asc_null_first", right_schema)?,
options: SortOptions {
descending: false,
nulls_first: true,
},
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("left", DataType::Int32, true),
Field::new("right", DataType::Int32, true),
]);
let filter_expr = join_expr_tests_fixture_i32(
case_expr,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
);
let column_indices = vec![
ColumnIndex {
index: 6,
side: JoinSide::Left,
},
ColumnIndex {
index: 6,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[tokio::test(flavor = "multi_thread")]
async fn build_null_columns_last() -> Result<()> {
let join_type = JoinType::Full;
let cardinality = (10, 11);
let case_expr = 1;
let config = SessionConfig::new().with_repartition_joins(false);
let session_ctx = SessionContext::with_config(config);
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let left_sorted = vec![PhysicalSortExpr {
expr: col("l_asc_null_last", left_schema)?,
options: SortOptions {
descending: false,
nulls_first: false,
},
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("r_asc_null_last", right_schema)?,
options: SortOptions {
descending: false,
nulls_first: false,
},
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("left", DataType::Int32, true),
Field::new("right", DataType::Int32, true),
]);
let filter_expr = join_expr_tests_fixture_i32(
case_expr,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
);
let column_indices = vec![
ColumnIndex {
index: 7,
side: JoinSide::Left,
},
ColumnIndex {
index: 7,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[tokio::test(flavor = "multi_thread")]
async fn build_null_columns_first_descending() -> Result<()> {
let join_type = JoinType::Full;
let cardinality = (10, 11);
let case_expr = 1;
let config = SessionConfig::new().with_repartition_joins(false);
let session_ctx = SessionContext::with_config(config);
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let left_sorted = vec![PhysicalSortExpr {
expr: col("l_desc_null_first", left_schema)?,
options: SortOptions {
descending: true,
nulls_first: true,
},
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("r_desc_null_first", right_schema)?,
options: SortOptions {
descending: true,
nulls_first: true,
},
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("left", DataType::Int32, true),
Field::new("right", DataType::Int32, true),
]);
let filter_expr = join_expr_tests_fixture_i32(
case_expr,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
);
let column_indices = vec![
ColumnIndex {
index: 8,
side: JoinSide::Left,
},
ColumnIndex {
index: 8,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[tokio::test(flavor = "multi_thread")]
async fn complex_join_all_one_ascending_numeric_missing_stat() -> Result<()> {
let cardinality = (3, 4);
let join_type = JoinType::Full;
// a + b > c + 10 AND a + b < c + 100
let config = SessionConfig::new().with_repartition_joins(false);
let session_ctx = SessionContext::with_config(config);
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let left_sorted = vec![PhysicalSortExpr {
expr: col("la1", left_schema)?,
options: SortOptions::default(),
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("ra1", right_schema)?,
options: SortOptions::default(),
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("0", DataType::Int32, true),
Field::new("1", DataType::Int32, true),
Field::new("2", DataType::Int32, true),
]);
let filter_expr = complicated_filter(&intermediate_schema)?;
let column_indices = vec![
ColumnIndex {
index: 0,
side: JoinSide::Left,
},
ColumnIndex {
index: 4,
side: JoinSide::Left,
},
ColumnIndex {
index: 0,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn test_one_side_hash_joiner_visited_rows(
#[values(
(JoinType::Inner, true),
(JoinType::Left,false),
(JoinType::Right, true),
(JoinType::RightSemi, true),
(JoinType::LeftSemi, false),
(JoinType::LeftAnti, false),
(JoinType::RightAnti, true),
(JoinType::Full, false),
)]
case: (JoinType, bool),
) -> Result<()> {
// Set a random state for the join
let join_type = case.0;
let should_be_empty = case.1;
let random_state = RandomState::with_seeds(0, 0, 0, 0);
let config = SessionConfig::new().with_repartition_joins(false);
let session_ctx = SessionContext::with_config(config);
let task_ctx = session_ctx.task_ctx();
// Ensure there will be matching rows
let (left_batch, right_batch) = build_sides_record_batches(20, (1, 1))?;
let left_schema = left_batch.schema();
let right_schema = right_batch.schema();
// Build the join schema from the left and right schemas
let (schema, join_column_indices) =
build_join_schema(&left_schema, &right_schema, &join_type);
let join_schema = Arc::new(schema);
// Sort information for MemoryExec
let left_sorted = vec![PhysicalSortExpr {
expr: col("la1", &left_schema)?,
options: SortOptions::default(),
}];
// Sort information for MemoryExec
let right_sorted = vec![PhysicalSortExpr {
expr: col("ra1", &right_schema)?,
options: SortOptions::default(),
}];
// Construct MemoryExec
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
10,
)?;
// Filter columns, ensure first batches will have matching rows.
let intermediate_schema = Schema::new(vec![
Field::new("0", DataType::Int32, true),
Field::new("1", DataType::Int32, true),
]);
let filter_expr = gen_conjunctive_numerical_expr(
col("0", &intermediate_schema)?,
col("1", &intermediate_schema)?,
(
Operator::Plus,
Operator::Minus,
Operator::Plus,
Operator::Plus,
),
ScalarValue::Int32(Some(0)),
ScalarValue::Int32(Some(3)),
ScalarValue::Int32(Some(0)),
ScalarValue::Int32(Some(3)),
(Operator::Gt, Operator::Lt),
);
let column_indices = vec![
ColumnIndex {
index: 0,
side: JoinSide::Left,
},
ColumnIndex {
index: 0,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
let mut left_side_joiner = OneSideHashJoiner::new(
JoinSide::Left,
vec![Column::new_with_schema("lc1", &left_schema)?],
left_schema,
);
let mut right_side_joiner = OneSideHashJoiner::new(
JoinSide::Right,
vec![Column::new_with_schema("rc1", &right_schema)?],
right_schema,
);
let mut left_stream = left.execute(0, task_ctx.clone())?;
let mut right_stream = right.execute(0, task_ctx)?;
let initial_left_batch = left_stream.next().await.unwrap()?;
left_side_joiner.update_internal_state(&initial_left_batch, &random_state)?;
assert_eq!(
left_side_joiner.input_buffer.num_rows(),
initial_left_batch.num_rows()
);
let initial_right_batch = right_stream.next().await.unwrap()?;
right_side_joiner.update_internal_state(&initial_right_batch, &random_state)?;
assert_eq!(
right_side_joiner.input_buffer.num_rows(),
initial_right_batch.num_rows()
);
left_side_joiner.join_with_probe_batch(
&join_schema,
join_type,
&right_side_joiner.on,
Some(&filter),
&initial_right_batch,
&mut right_side_joiner.visited_rows,
right_side_joiner.offset,
&join_column_indices,
&random_state,
false,
)?;
assert_eq!(left_side_joiner.visited_rows.is_empty(), should_be_empty);
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn testing_with_temporal_columns(
#[values(
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::RightSemi,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::Full
)]
join_type: JoinType,
#[values(
(4, 5),
(99, 12),
)]
cardinality: (i32, i32),
#[values(0, 1)] case_expr: usize,
) -> Result<()> {
let config = SessionConfig::new().with_repartition_joins(false);
let session_ctx = SessionContext::with_config(config);
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let left_sorted = vec![PhysicalSortExpr {
expr: col("lt1", left_schema)?,
options: SortOptions {
descending: false,
nulls_first: true,
},
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("rt1", right_schema)?,
options: SortOptions {
descending: false,
nulls_first: true,
},
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let intermediate_schema = Schema::new(vec![
Field::new(
"left",
DataType::Timestamp(TimeUnit::Millisecond, None),
false,
),
Field::new(
"right",
DataType::Timestamp(TimeUnit::Millisecond, None),
false,
),
]);
let filter_expr = join_expr_tests_fixture_temporal(
case_expr,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
&intermediate_schema,
)?;
let column_indices = vec![
ColumnIndex {
index: 3,
side: JoinSide::Left,
},
ColumnIndex {
index: 3,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn test_with_interval_columns(
#[values(
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::RightSemi,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::Full
)]
join_type: JoinType,
#[values(
(4, 5),
(99, 12),
)]
cardinality: (i32, i32),
) -> Result<()> {
let config = SessionConfig::new().with_repartition_joins(false);
let session_ctx = SessionContext::with_config(config);
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let left_sorted = vec![PhysicalSortExpr {
expr: col("li1", left_schema)?,
options: SortOptions {
descending: false,
nulls_first: true,
},
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("ri1", right_schema)?,
options: SortOptions {
descending: false,
nulls_first: true,
},
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let intermediate_schema = Schema::new(vec![
Field::new("left", DataType::Interval(IntervalUnit::DayTime), false),
Field::new("right", DataType::Interval(IntervalUnit::DayTime), false),
]);
let filter_expr = join_expr_tests_fixture_temporal(
0,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
&intermediate_schema,
)?;
let column_indices = vec![
ColumnIndex {
index: 9,
side: JoinSide::Left,
},
ColumnIndex {
index: 9,
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
#[rstest]
#[tokio::test(flavor = "multi_thread")]
async fn testing_ascending_float_pruning(
#[values(
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::RightSemi,
JoinType::LeftSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::Full
)]
join_type: JoinType,
#[values(
(4, 5),
(99, 12),
)]
cardinality: (i32, i32),
#[values(0, 1, 2, 3, 4, 5, 6, 7)] case_expr: usize,
) -> Result<()> {
let config = SessionConfig::new().with_repartition_joins(false);
let session_ctx = SessionContext::with_config(config);
let task_ctx = session_ctx.task_ctx();
let (left_batch, right_batch) =
build_sides_record_batches(TABLE_SIZE, cardinality)?;
let left_schema = &left_batch.schema();
let right_schema = &right_batch.schema();
let left_sorted = vec![PhysicalSortExpr {
expr: col("l_float", left_schema)?,
options: SortOptions::default(),
}];
let right_sorted = vec![PhysicalSortExpr {
expr: col("r_float", right_schema)?,
options: SortOptions::default(),
}];
let (left, right) = create_memory_table(
left_batch,
right_batch,
Some(left_sorted),
Some(right_sorted),
13,
)?;
let on = vec![(
Column::new_with_schema("lc1", left_schema)?,
Column::new_with_schema("rc1", right_schema)?,
)];
let intermediate_schema = Schema::new(vec![
Field::new("left", DataType::Float64, true),
Field::new("right", DataType::Float64, true),
]);
let filter_expr = join_expr_tests_fixture_f64(
case_expr,
col("left", &intermediate_schema)?,
col("right", &intermediate_schema)?,
);
let column_indices = vec![
ColumnIndex {
index: 10, // l_float
side: JoinSide::Left,
},
ColumnIndex {
index: 10, // r_float
side: JoinSide::Right,
},
];
let filter = JoinFilter::new(filter_expr, column_indices, intermediate_schema);
experiment(left, right, Some(filter), join_type, on, task_ctx).await?;
Ok(())
}
}