datafusion/dataframe/mod.rs
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162 1163 1164 1165 1166 1167 1168 1169 1170 1171 1172 1173 1174 1175 1176 1177 1178 1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255 1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361 1362 1363 1364 1365 1366 1367 1368 1369 1370 1371 1372 1373 1374 1375 1376 1377 1378 1379 1380 1381 1382 1383 1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397 1398 1399 1400 1401 1402 1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418 1419 1420 1421 1422 1423 1424 1425 1426 1427 1428 1429 1430 1431 1432 1433 1434 1435 1436 1437 1438 1439 1440 1441 1442 1443 1444 1445 1446 1447 1448 1449 1450 1451 1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466 1467 1468 1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483 1484 1485 1486 1487 1488 1489 1490 1491 1492 1493 1494 1495 1496 1497 1498 1499 1500 1501 1502 1503 1504 1505 1506 1507 1508 1509 1510 1511 1512 1513 1514 1515 1516 1517 1518 1519 1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564 1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578 1579 1580 1581 1582 1583 1584 1585 1586 1587 1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616 1617 1618 1619 1620 1621 1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633 1634 1635 1636 1637 1638 1639 1640 1641 1642 1643 1644 1645 1646 1647 1648 1649 1650 1651 1652 1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667 1668 1669 1670 1671 1672 1673 1674 1675 1676 1677 1678 1679 1680 1681 1682 1683 1684 1685 1686 1687 1688 1689 1690 1691 1692 1693 1694 1695 1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707 1708 1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719 1720 1721 1722 1723 1724 1725 1726 1727 1728 1729 1730 1731 1732 1733 1734 1735 1736 1737 1738 1739 1740 1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778 1779 1780 1781 1782 1783 1784 1785 1786 1787 1788 1789 1790 1791 1792 1793 1794 1795 1796 1797 1798 1799 1800 1801 1802 1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813 1814 1815 1816 1817 1818 1819 1820 1821 1822 1823 1824 1825 1826 1827 1828 1829 1830 1831 1832 1833 1834 1835 1836 1837 1838 1839 1840 1841 1842 1843 1844 1845 1846 1847 1848 1849 1850 1851 1852 1853 1854 1855 1856 1857 1858 1859 1860 1861 1862 1863 1864 1865 1866 1867 1868 1869 1870 1871 1872 1873 1874 1875 1876 1877 1878 1879 1880 1881 1882 1883 1884 1885 1886 1887 1888 1889 1890 1891 1892 1893 1894 1895 1896 1897 1898 1899 1900 1901 1902 1903 1904 1905 1906 1907 1908 1909 1910 1911 1912 1913 1914 1915 1916 1917 1918 1919 1920 1921 1922 1923 1924 1925 1926 1927 1928 1929 1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941 1942 1943 1944 1945 1946 1947 1948 1949 1950 1951 1952 1953 1954 1955 1956 1957 1958 1959 1960 1961 1962 1963 1964 1965 1966 1967 1968 1969 1970 1971 1972 1973 1974 1975 1976 1977 1978 1979 1980 1981 1982 1983 1984 1985 1986 1987 1988 1989 1990 1991 1992 1993 1994 1995 1996 1997 1998 1999 2000 2001 2002 2003 2004 2005 2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029 2030 2031 2032 2033 2034 2035 2036 2037 2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050 2051 2052 2053 2054 2055 2056 2057 2058 2059 2060 2061 2062 2063 2064 2065 2066 2067 2068 2069 2070 2071 2072 2073 2074 2075 2076 2077 2078 2079 2080 2081 2082 2083 2084 2085 2086 2087 2088 2089 2090 2091 2092 2093 2094 2095 2096 2097 2098 2099 2100 2101 2102 2103 2104 2105 2106 2107 2108 2109 2110 2111 2112 2113 2114 2115 2116 2117 2118 2119 2120 2121 2122 2123 2124 2125 2126 2127 2128 2129 2130 2131 2132 2133 2134 2135 2136 2137 2138 2139 2140 2141 2142 2143 2144 2145 2146 2147 2148 2149 2150 2151 2152 2153 2154 2155 2156 2157 2158 2159 2160 2161 2162 2163 2164 2165 2166 2167 2168 2169 2170 2171 2172 2173 2174 2175 2176 2177 2178 2179 2180 2181 2182 2183 2184 2185 2186 2187 2188 2189 2190 2191 2192 2193 2194 2195 2196 2197 2198 2199 2200 2201 2202 2203 2204 2205 2206 2207 2208 2209 2210 2211 2212 2213 2214 2215 2216 2217 2218 2219 2220 2221 2222 2223 2224 2225 2226 2227 2228 2229 2230 2231 2232 2233 2234 2235 2236 2237 2238 2239 2240 2241 2242 2243 2244 2245 2246 2247 2248 2249 2250 2251 2252 2253 2254 2255 2256 2257 2258 2259 2260 2261 2262 2263 2264 2265 2266 2267 2268 2269 2270 2271 2272 2273 2274 2275 2276 2277 2278 2279 2280 2281 2282 2283 2284 2285 2286 2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301 2302 2303 2304 2305 2306 2307 2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326 2327 2328 2329 2330 2331 2332 2333 2334 2335 2336 2337 2338 2339 2340 2341 2342 2343 2344 2345 2346 2347 2348 2349 2350 2351 2352 2353 2354 2355 2356 2357 2358 2359 2360 2361 2362 2363 2364 2365 2366 2367 2368 2369 2370 2371 2372 2373 2374 2375 2376 2377 2378 2379 2380 2381 2382 2383 2384 2385 2386 2387 2388 2389 2390 2391 2392 2393 2394 2395 2396 2397 2398 2399 2400 2401 2402 2403 2404 2405 2406 2407 2408 2409 2410 2411 2412 2413 2414 2415 2416 2417 2418 2419 2420 2421 2422 2423 2424 2425 2426 2427 2428 2429 2430 2431 2432 2433 2434 2435 2436 2437 2438 2439 2440 2441 2442 2443 2444 2445 2446 2447 2448 2449 2450 2451 2452 2453 2454 2455 2456 2457 2458 2459 2460 2461 2462 2463 2464 2465 2466 2467 2468 2469 2470 2471 2472 2473 2474 2475 2476 2477 2478 2479 2480 2481 2482 2483 2484 2485 2486 2487 2488 2489 2490 2491 2492 2493 2494 2495 2496 2497 2498 2499 2500 2501 2502 2503 2504 2505 2506 2507 2508 2509 2510 2511 2512 2513 2514 2515 2516 2517 2518 2519 2520 2521 2522 2523 2524 2525 2526 2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537 2538 2539 2540 2541 2542 2543 2544 2545 2546 2547 2548 2549 2550 2551 2552 2553 2554 2555 2556 2557 2558 2559 2560 2561 2562 2563 2564 2565 2566 2567 2568 2569 2570 2571 2572 2573 2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584 2585 2586 2587 2588 2589 2590 2591 2592 2593 2594 2595 2596 2597 2598 2599 2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623 2624 2625 2626 2627 2628 2629 2630 2631 2632 2633 2634 2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652 2653 2654 2655 2656 2657 2658 2659 2660 2661 2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672 2673 2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684 2685 2686 2687 2688 2689 2690 2691 2692 2693 2694 2695 2696 2697 2698 2699 2700 2701 2702 2703 2704 2705 2706 2707 2708 2709 2710 2711 2712 2713 2714 2715 2716 2717 2718 2719 2720 2721 2722 2723 2724 2725 2726 2727 2728 2729 2730 2731 2732 2733 2734 2735 2736 2737 2738 2739 2740 2741 2742 2743 2744 2745 2746 2747 2748 2749 2750 2751 2752 2753 2754 2755 2756 2757 2758 2759 2760 2761 2762 2763 2764 2765 2766 2767 2768 2769 2770 2771 2772 2773 2774 2775 2776 2777 2778 2779 2780 2781 2782 2783 2784 2785 2786 2787 2788 2789 2790 2791 2792 2793 2794 2795 2796 2797 2798 2799 2800 2801 2802 2803 2804 2805 2806 2807 2808 2809 2810 2811 2812 2813 2814 2815 2816 2817 2818 2819 2820 2821 2822 2823 2824 2825 2826 2827 2828 2829 2830 2831 2832 2833 2834 2835 2836 2837 2838 2839 2840 2841 2842 2843 2844 2845 2846 2847 2848 2849 2850 2851 2852 2853 2854 2855 2856 2857 2858 2859 2860 2861 2862 2863 2864 2865 2866 2867 2868 2869 2870 2871 2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884 2885 2886 2887 2888 2889 2890 2891 2892 2893 2894 2895 2896 2897 2898 2899 2900 2901 2902 2903 2904 2905 2906 2907 2908 2909 2910 2911 2912 2913 2914 2915 2916 2917 2918 2919 2920 2921 2922 2923 2924 2925 2926 2927 2928 2929 2930 2931 2932 2933 2934 2935 2936 2937 2938 2939 2940 2941 2942 2943 2944 2945 2946 2947 2948 2949 2950 2951 2952 2953 2954 2955 2956 2957 2958 2959 2960 2961 2962 2963 2964 2965 2966 2967 2968 2969 2970 2971 2972 2973 2974 2975 2976 2977 2978 2979 2980 2981 2982 2983 2984 2985 2986 2987 2988 2989 2990 2991 2992 2993 2994 2995 2996 2997 2998 2999 3000 3001 3002 3003 3004 3005 3006 3007 3008 3009 3010 3011 3012 3013 3014 3015 3016 3017 3018 3019 3020 3021 3022 3023 3024 3025 3026 3027 3028 3029 3030 3031 3032 3033 3034 3035 3036 3037 3038 3039 3040 3041 3042 3043 3044 3045 3046 3047 3048 3049 3050 3051 3052 3053 3054 3055 3056 3057 3058 3059 3060 3061 3062 3063 3064 3065 3066 3067 3068 3069 3070 3071 3072 3073 3074 3075 3076 3077 3078 3079 3080 3081 3082 3083 3084 3085 3086 3087 3088 3089 3090 3091 3092 3093 3094 3095 3096 3097 3098 3099 3100 3101 3102 3103 3104 3105 3106 3107 3108 3109 3110 3111 3112 3113 3114 3115 3116 3117 3118 3119 3120 3121 3122 3123 3124 3125 3126 3127 3128 3129 3130 3131 3132 3133 3134 3135 3136 3137 3138 3139 3140 3141 3142 3143 3144 3145 3146 3147 3148 3149 3150 3151 3152 3153 3154 3155 3156 3157 3158 3159 3160 3161 3162 3163 3164 3165 3166 3167 3168 3169 3170 3171 3172 3173 3174 3175 3176 3177 3178 3179 3180 3181 3182 3183 3184 3185 3186 3187 3188 3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200 3201 3202 3203 3204 3205 3206 3207 3208 3209 3210 3211 3212 3213 3214 3215 3216 3217 3218 3219 3220 3221 3222 3223 3224 3225 3226 3227 3228 3229 3230 3231 3232 3233 3234 3235 3236 3237 3238 3239 3240 3241 3242 3243 3244 3245 3246 3247 3248 3249 3250 3251 3252 3253 3254 3255 3256 3257 3258 3259 3260 3261 3262 3263 3264 3265 3266 3267 3268 3269 3270 3271 3272 3273 3274 3275 3276 3277 3278 3279 3280 3281 3282 3283 3284 3285 3286 3287 3288 3289 3290 3291 3292 3293 3294 3295 3296 3297 3298 3299 3300 3301 3302 3303 3304 3305 3306 3307 3308 3309 3310 3311 3312 3313 3314 3315 3316 3317 3318 3319 3320 3321 3322 3323 3324 3325 3326 3327 3328 3329 3330 3331 3332 3333 3334 3335 3336 3337 3338 3339 3340 3341 3342 3343 3344 3345 3346 3347 3348 3349 3350 3351 3352 3353 3354 3355 3356 3357 3358 3359 3360 3361 3362 3363 3364 3365 3366 3367 3368 3369 3370 3371 3372 3373 3374 3375 3376 3377 3378 3379 3380 3381 3382 3383 3384 3385 3386 3387 3388 3389 3390 3391 3392 3393 3394 3395 3396 3397 3398 3399 3400 3401 3402 3403 3404 3405 3406 3407 3408 3409 3410 3411 3412 3413 3414 3415 3416 3417 3418 3419 3420 3421 3422 3423 3424 3425 3426 3427 3428 3429 3430 3431 3432 3433 3434 3435 3436 3437 3438 3439 3440 3441 3442 3443 3444 3445 3446 3447 3448 3449 3450 3451 3452 3453 3454 3455 3456 3457 3458 3459 3460 3461 3462 3463 3464 3465 3466 3467 3468 3469 3470 3471 3472 3473 3474 3475 3476 3477 3478 3479 3480 3481 3482 3483 3484 3485 3486 3487 3488 3489 3490 3491 3492 3493 3494 3495 3496 3497 3498 3499 3500 3501 3502 3503 3504 3505 3506 3507 3508 3509 3510 3511 3512 3513 3514 3515 3516 3517 3518 3519 3520 3521 3522 3523 3524 3525 3526 3527 3528 3529 3530 3531 3532 3533 3534 3535 3536 3537 3538 3539 3540 3541 3542 3543 3544 3545 3546 3547 3548 3549 3550 3551 3552 3553 3554 3555 3556 3557 3558 3559 3560 3561 3562 3563 3564 3565 3566 3567 3568 3569 3570 3571 3572 3573 3574 3575 3576 3577 3578 3579 3580 3581 3582 3583 3584 3585 3586 3587 3588 3589 3590 3591 3592 3593 3594 3595 3596 3597 3598 3599 3600 3601 3602 3603 3604 3605 3606 3607 3608 3609 3610 3611 3612 3613 3614 3615 3616 3617 3618 3619 3620 3621 3622 3623 3624 3625 3626 3627 3628 3629 3630 3631 3632 3633 3634 3635 3636 3637 3638 3639 3640 3641 3642 3643 3644 3645 3646 3647 3648 3649 3650 3651 3652 3653 3654 3655 3656 3657 3658 3659 3660 3661 3662 3663 3664 3665 3666 3667 3668 3669 3670 3671 3672 3673 3674 3675 3676 3677 3678 3679 3680 3681 3682 3683 3684 3685 3686 3687 3688 3689 3690 3691 3692 3693 3694 3695 3696 3697 3698 3699 3700 3701 3702 3703 3704 3705 3706 3707 3708 3709 3710 3711 3712 3713 3714 3715 3716 3717 3718 3719 3720 3721 3722 3723 3724 3725 3726 3727 3728 3729 3730 3731 3732 3733 3734 3735 3736 3737 3738 3739 3740 3741 3742 3743 3744 3745 3746 3747 3748 3749 3750 3751 3752 3753 3754 3755 3756 3757 3758 3759 3760 3761 3762 3763 3764 3765 3766 3767 3768 3769 3770 3771 3772 3773 3774 3775 3776 3777 3778 3779 3780 3781 3782 3783 3784 3785 3786 3787 3788 3789 3790 3791 3792 3793 3794 3795 3796 3797 3798 3799 3800 3801 3802 3803 3804 3805 3806 3807 3808 3809 3810 3811 3812 3813 3814 3815 3816 3817 3818 3819 3820 3821 3822 3823 3824 3825 3826 3827 3828 3829 3830 3831 3832 3833 3834 3835 3836 3837 3838 3839 3840 3841 3842 3843 3844 3845 3846 3847 3848 3849 3850 3851 3852 3853 3854 3855 3856 3857 3858 3859 3860 3861 3862 3863 3864 3865 3866 3867 3868 3869 3870 3871 3872 3873 3874 3875 3876 3877 3878 3879 3880 3881 3882 3883 3884 3885 3886 3887 3888 3889 3890 3891 3892 3893 3894 3895 3896 3897 3898 3899 3900 3901 3902 3903 3904 3905 3906 3907 3908 3909 3910 3911 3912 3913 3914 3915 3916 3917 3918 3919 3920 3921 3922 3923 3924 3925 3926 3927 3928 3929 3930 3931 3932 3933 3934 3935 3936 3937 3938 3939 3940 3941 3942 3943 3944 3945 3946 3947 3948 3949 3950 3951 3952 3953 3954 3955 3956 3957 3958 3959 3960 3961 3962 3963 3964 3965 3966 3967 3968 3969 3970 3971 3972 3973 3974 3975 3976 3977 3978 3979 3980 3981 3982 3983 3984 3985 3986 3987 3988 3989 3990 3991 3992 3993 3994 3995 3996 3997 3998 3999 4000 4001 4002 4003 4004 4005 4006 4007 4008 4009 4010 4011 4012 4013 4014 4015 4016 4017 4018 4019 4020 4021 4022 4023 4024 4025 4026 4027 4028 4029 4030 4031 4032 4033 4034 4035 4036 4037 4038 4039 4040 4041 4042 4043 4044 4045 4046 4047 4048 4049 4050 4051 4052 4053 4054 4055 4056 4057 4058 4059 4060 4061 4062 4063 4064 4065 4066 4067 4068 4069 4070 4071 4072 4073 4074 4075 4076 4077 4078 4079 4080 4081 4082 4083 4084 4085 4086 4087 4088 4089 4090 4091 4092 4093 4094 4095 4096 4097 4098 4099 4100 4101 4102 4103 4104 4105 4106 4107 4108 4109 4110 4111 4112 4113 4114 4115 4116 4117 4118 4119 4120 4121 4122 4123 4124 4125 4126 4127 4128 4129 4130 4131 4132 4133 4134 4135 4136 4137 4138 4139 4140 4141 4142 4143 4144 4145 4146 4147 4148 4149 4150 4151 4152 4153 4154 4155 4156 4157 4158 4159 4160 4161 4162 4163 4164 4165 4166 4167 4168 4169 4170 4171 4172 4173 4174 4175 4176 4177 4178 4179 4180 4181 4182 4183 4184 4185 4186 4187 4188 4189 4190 4191 4192 4193 4194 4195 4196 4197 4198 4199 4200 4201 4202 4203 4204 4205 4206 4207 4208 4209 4210 4211 4212 4213 4214 4215 4216 4217 4218 4219 4220 4221 4222 4223 4224 4225 4226 4227 4228 4229 4230 4231 4232 4233 4234 4235 4236 4237 4238 4239 4240 4241 4242 4243 4244 4245 4246 4247 4248 4249 4250 4251 4252 4253 4254 4255 4256 4257 4258 4259 4260 4261 4262 4263 4264 4265 4266 4267 4268 4269 4270 4271 4272 4273 4274 4275 4276 4277 4278 4279 4280 4281 4282 4283 4284 4285 4286 4287 4288 4289 4290 4291 4292 4293 4294 4295 4296 4297 4298 4299 4300 4301 4302 4303 4304 4305 4306 4307 4308 4309 4310 4311 4312 4313 4314 4315 4316 4317 4318 4319 4320 4321 4322 4323 4324 4325 4326 4327 4328 4329 4330
// 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.
//! [`DataFrame`] API for building and executing query plans.
#[cfg(feature = "parquet")]
mod parquet;
use std::any::Any;
use std::borrow::Cow;
use std::collections::HashMap;
use std::sync::Arc;
use crate::arrow::record_batch::RecordBatch;
use crate::arrow::util::pretty;
use crate::datasource::file_format::csv::CsvFormatFactory;
use crate::datasource::file_format::format_as_file_type;
use crate::datasource::file_format::json::JsonFormatFactory;
use crate::datasource::{provider_as_source, MemTable, TableProvider};
use crate::error::Result;
use crate::execution::context::{SessionState, TaskContext};
use crate::execution::FunctionRegistry;
use crate::logical_expr::utils::find_window_exprs;
use crate::logical_expr::{
col, Expr, JoinType, LogicalPlan, LogicalPlanBuilder, Partitioning, TableType,
};
use crate::physical_plan::{
collect, collect_partitioned, execute_stream, execute_stream_partitioned,
ExecutionPlan, SendableRecordBatchStream,
};
use crate::prelude::SessionContext;
use arrow::array::{Array, ArrayRef, Int64Array, StringArray};
use arrow::compute::{cast, concat};
use arrow::datatypes::{DataType, Field};
use arrow_schema::{Schema, SchemaRef};
use datafusion_common::config::{CsvOptions, JsonOptions};
use datafusion_common::{
exec_err, not_impl_err, plan_err, Column, DFSchema, DataFusionError, ParamValues,
SchemaError, UnnestOptions,
};
use datafusion_expr::dml::InsertOp;
use datafusion_expr::{case, is_null, lit, SortExpr};
use datafusion_expr::{
utils::COUNT_STAR_EXPANSION, TableProviderFilterPushDown, UNNAMED_TABLE,
};
use datafusion_functions_aggregate::expr_fn::{
avg, count, max, median, min, stddev, sum,
};
use async_trait::async_trait;
use datafusion_catalog::Session;
/// Contains options that control how data is
/// written out from a DataFrame
pub struct DataFrameWriteOptions {
/// Controls how new data should be written to the table, determining whether
/// to append, overwrite, or replace existing data.
insert_op: InsertOp,
/// Controls if all partitions should be coalesced into a single output file
/// Generally will have slower performance when set to true.
single_file_output: bool,
/// Sets which columns should be used for hive-style partitioned writes by name.
/// Can be set to empty vec![] for non-partitioned writes.
partition_by: Vec<String>,
/// Sets which columns should be used for sorting the output by name.
/// Can be set to empty vec![] for non-sorted writes.
sort_by: Vec<SortExpr>,
}
impl DataFrameWriteOptions {
/// Create a new DataFrameWriteOptions with default values
pub fn new() -> Self {
DataFrameWriteOptions {
insert_op: InsertOp::Append,
single_file_output: false,
partition_by: vec![],
sort_by: vec![],
}
}
/// Set the insert operation
pub fn with_insert_operation(mut self, insert_op: InsertOp) -> Self {
self.insert_op = insert_op;
self
}
/// Set the single_file_output value to true or false
pub fn with_single_file_output(mut self, single_file_output: bool) -> Self {
self.single_file_output = single_file_output;
self
}
/// Sets the partition_by columns for output partitioning
pub fn with_partition_by(mut self, partition_by: Vec<String>) -> Self {
self.partition_by = partition_by;
self
}
/// Sets the sort_by columns for output sorting
pub fn with_sort_by(mut self, sort_by: Vec<SortExpr>) -> Self {
self.sort_by = sort_by;
self
}
}
impl Default for DataFrameWriteOptions {
fn default() -> Self {
Self::new()
}
}
/// Represents a logical set of rows with the same named columns.
///
/// Similar to a [Pandas DataFrame] or [Spark DataFrame], a DataFusion DataFrame
/// represents a 2 dimensional table of rows and columns.
///
/// The typical workflow using DataFrames looks like
///
/// 1. Create a DataFrame via methods on [SessionContext], such as [`read_csv`]
/// and [`read_parquet`].
///
/// 2. Build a desired calculation by calling methods such as [`filter`],
/// [`select`], [`aggregate`], and [`limit`]
///
/// 3. Execute into [`RecordBatch`]es by calling [`collect`]
///
/// A `DataFrame` is a wrapper around a [`LogicalPlan`] and the [`SessionState`]
/// required for execution.
///
/// DataFrames are "lazy" in the sense that most methods do not actually compute
/// anything, they just build up a plan. Calling [`collect`] executes the plan
/// using the same DataFusion planning and execution process used to execute SQL
/// and other queries.
///
/// [Pandas DataFrame]: https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html
/// [Spark DataFrame]: https://spark.apache.org/docs/latest/sql-programming-guide.html
/// [`read_csv`]: SessionContext::read_csv
/// [`read_parquet`]: SessionContext::read_parquet
/// [`filter`]: DataFrame::filter
/// [`select`]: DataFrame::select
/// [`aggregate`]: DataFrame::aggregate
/// [`limit`]: DataFrame::limit
/// [`collect`]: DataFrame::collect
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion::functions_aggregate::expr_fn::min;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// // Read the data from a csv file
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// // create a new dataframe that computes the equivalent of
/// // `SELECT a, MIN(b) FROM df WHERE a <= b GROUP BY a LIMIT 100;`
/// let df = df.filter(col("a").lt_eq(col("b")))?
/// .aggregate(vec![col("a")], vec![min(col("b"))])?
/// .limit(0, Some(100))?;
/// // Perform the actual computation
/// let results = df.collect();
/// # Ok(())
/// # }
/// ```
#[derive(Debug, Clone)]
pub struct DataFrame {
// Box the (large) SessionState to reduce the size of DataFrame on the stack
session_state: Box<SessionState>,
plan: LogicalPlan,
}
impl DataFrame {
/// Create a new `DataFrame ` based on an existing `LogicalPlan`
///
/// This is a low-level method and is not typically used by end users. See
/// [`SessionContext::read_csv`] and other methods for creating a
/// `DataFrame` from an existing datasource.
pub fn new(session_state: SessionState, plan: LogicalPlan) -> Self {
Self {
session_state: Box::new(session_state),
plan,
}
}
/// Creates logical expression from a SQL query text.
/// The expression is created and processed against the current schema.
///
/// # Example: Parsing SQL queries
/// ```
/// # use arrow::datatypes::{DataType, Field, Schema};
/// # use datafusion::prelude::*;
/// # use datafusion_common::{DFSchema, Result};
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// // datafusion will parse number as i64 first.
/// let sql = "a > 1 and b in (1, 10)";
/// let expected = col("a").gt(lit(1 as i64))
/// .and(col("b").in_list(vec![lit(1 as i64), lit(10 as i64)], false));
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let expr = df.parse_sql_expr(sql)?;
/// assert_eq!(expected, expr);
/// # Ok(())
/// # }
/// ```
pub fn parse_sql_expr(&self, sql: &str) -> Result<Expr> {
let df_schema = self.schema();
self.session_state.create_logical_expr(sql, df_schema)
}
/// Consume the DataFrame and produce a physical plan
pub async fn create_physical_plan(self) -> Result<Arc<dyn ExecutionPlan>> {
self.session_state.create_physical_plan(&self.plan).await
}
/// Filter the DataFrame by column. Returns a new DataFrame only containing the
/// specified columns.
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.select_columns(&["a", "b"])?;
/// let expected = vec![
/// "+---+---+",
/// "| a | b |",
/// "+---+---+",
/// "| 1 | 2 |",
/// "+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn select_columns(self, columns: &[&str]) -> Result<DataFrame> {
let fields = columns
.iter()
.map(|name| {
self.plan
.schema()
.qualified_field_with_unqualified_name(name)
})
.collect::<Result<Vec<_>>>()?;
let expr: Vec<Expr> = fields
.into_iter()
.map(|(qualifier, field)| Expr::Column(Column::from((qualifier, field))))
.collect();
self.select(expr)
}
/// Project arbitrary list of expression strings into a new `DataFrame`.
/// Method will parse string expressions into logical plan expressions.
///
/// The output `DataFrame` has one column for each element in `exprs`.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df : DataFrame = df.select_exprs(&["a * b", "c"])?;
/// # Ok(())
/// # }
/// ```
pub fn select_exprs(self, exprs: &[&str]) -> Result<DataFrame> {
let expr_list = exprs
.iter()
.map(|e| self.parse_sql_expr(e))
.collect::<Result<Vec<_>>>()?;
self.select(expr_list)
}
/// Project arbitrary expressions (like SQL SELECT expressions) into a new
/// `DataFrame`.
///
/// The output `DataFrame` has one column for each element in `expr_list`.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.select(vec![col("a"), col("b") * col("c")])?;
/// let expected = vec![
/// "+---+-----------------------+",
/// "| a | ?table?.b * ?table?.c |",
/// "+---+-----------------------+",
/// "| 1 | 6 |",
/// "+---+-----------------------+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn select(self, expr_list: Vec<Expr>) -> Result<DataFrame> {
let window_func_exprs = find_window_exprs(&expr_list);
let plan = if window_func_exprs.is_empty() {
self.plan
} else {
LogicalPlanBuilder::window_plan(self.plan, window_func_exprs)?
};
let project_plan = LogicalPlanBuilder::from(plan).project(expr_list)?.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan: project_plan,
})
}
/// Returns a new DataFrame containing all columns except the specified columns.
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// // +----+----+----+
/// // | a | b | c |
/// // +----+----+----+
/// // | 1 | 2 | 3 |
/// // +----+----+----+
/// let df = df.drop_columns(&["a"])?;
/// let expected = vec![
/// "+---+---+",
/// "| b | c |",
/// "+---+---+",
/// "| 2 | 3 |",
/// "+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn drop_columns(self, columns: &[&str]) -> Result<DataFrame> {
let fields_to_drop = columns
.iter()
.map(|name| {
self.plan
.schema()
.qualified_field_with_unqualified_name(name)
})
.filter(|r| r.is_ok())
.collect::<Result<Vec<_>>>()?;
let expr: Vec<Expr> = self
.plan
.schema()
.fields()
.into_iter()
.enumerate()
.map(|(idx, _)| self.plan.schema().qualified_field(idx))
.filter(|(qualifier, f)| !fields_to_drop.contains(&(*qualifier, f)))
.map(|(qualifier, field)| Expr::Column(Column::from((qualifier, field))))
.collect();
self.select(expr)
}
/// Expand multiple list/struct columns into a set of rows and new columns.
///
/// See also: [`UnnestOptions`] documentation for the behavior of `unnest`
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_json("tests/data/unnest.json", NdJsonReadOptions::default()).await?;
/// // expand into multiple columns if it's json array, flatten field name if it's nested structure
/// let df = df.unnest_columns(&["b","c","d"])?;
/// let expected = vec![
/// "+---+------+-------+-----+-----+",
/// "| a | b | c | d.e | d.f |",
/// "+---+------+-------+-----+-----+",
/// "| 1 | 2.0 | false | 1 | 2 |",
/// "| 1 | 1.3 | true | 1 | 2 |",
/// "| 1 | -6.1 | | 1 | 2 |",
/// "| 2 | 3.0 | false | | |",
/// "| 2 | 2.3 | true | | |",
/// "| 2 | -7.1 | | | |",
/// "+---+------+-------+-----+-----+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn unnest_columns(self, columns: &[&str]) -> Result<DataFrame> {
self.unnest_columns_with_options(columns, UnnestOptions::new())
}
/// Expand multiple list columns into a set of rows, with
/// behavior controlled by [`UnnestOptions`].
///
/// Please see the documentation on [`UnnestOptions`] for more
/// details about the meaning of unnest.
pub fn unnest_columns_with_options(
self,
columns: &[&str],
options: UnnestOptions,
) -> Result<DataFrame> {
let columns = columns.iter().map(|c| Column::from(*c)).collect();
let plan = LogicalPlanBuilder::from(self.plan)
.unnest_columns_with_options(columns, options)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a DataFrame with only rows for which `predicate` evaluates to
/// `true`.
///
/// Rows for which `predicate` evaluates to `false` or `null`
/// are filtered out.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?;
/// let df = df.filter(col("a").lt_eq(col("b")))?;
/// // all rows where a <= b are returned
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 1 | 2 | 3 |",
/// "| 4 | 5 | 6 |",
/// "| 7 | 8 | 9 |",
/// "+---+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn filter(self, predicate: Expr) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.filter(predicate)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new `DataFrame` that aggregates the rows of the current
/// `DataFrame`, first optionally grouping by the given expressions.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion::functions_aggregate::expr_fn::min;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?;
///
/// // The following use is the equivalent of "SELECT MIN(b) GROUP BY a"
/// let df1 = df.clone().aggregate(vec![col("a")], vec![min(col("b"))])?;
/// let expected1 = vec![
/// "+---+----------------+",
/// "| a | min(?table?.b) |",
/// "+---+----------------+",
/// "| 1 | 2 |",
/// "| 4 | 5 |",
/// "| 7 | 8 |",
/// "+---+----------------+"
/// ];
/// assert_batches_sorted_eq!(expected1, &df1.collect().await?);
/// // The following use is the equivalent of "SELECT MIN(b)"
/// let df2 = df.aggregate(vec![], vec![min(col("b"))])?;
/// let expected2 = vec![
/// "+----------------+",
/// "| min(?table?.b) |",
/// "+----------------+",
/// "| 2 |",
/// "+----------------+"
/// ];
/// # assert_batches_sorted_eq!(expected2, &df2.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn aggregate(
self,
group_expr: Vec<Expr>,
aggr_expr: Vec<Expr>,
) -> Result<DataFrame> {
let is_grouping_set = matches!(group_expr.as_slice(), [Expr::GroupingSet(_)]);
let aggr_expr_len = aggr_expr.len();
let plan = LogicalPlanBuilder::from(self.plan)
.aggregate(group_expr, aggr_expr)?
.build()?;
let plan = if is_grouping_set {
let grouping_id_pos = plan.schema().fields().len() - 1 - aggr_expr_len;
// For grouping sets we do a project to not expose the internal grouping id
let exprs = plan
.schema()
.columns()
.into_iter()
.enumerate()
.filter(|(idx, _)| *idx != grouping_id_pos)
.map(|(_, column)| Expr::Column(column))
.collect::<Vec<_>>();
LogicalPlanBuilder::from(plan).project(exprs)?.build()?
} else {
plan
};
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new DataFrame that adds the result of evaluating one or more
/// window functions ([`Expr::WindowFunction`]) to the existing columns
pub fn window(self, window_exprs: Vec<Expr>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.window(window_exprs)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Returns a new `DataFrame` with a limited number of rows.
///
/// # Arguments
/// `skip` - Number of rows to skip before fetch any row
/// `fetch` - Maximum number of rows to return, after skipping `skip` rows.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?;
/// let df = df.limit(1, Some(2))?;
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 4 | 5 | 6 |",
/// "| 7 | 8 | 9 |",
/// "+---+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn limit(self, skip: usize, fetch: Option<usize>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.limit(skip, fetch)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Calculate the union of two [`DataFrame`]s, preserving duplicate rows.
///
/// The two [`DataFrame`]s must have exactly the same schema
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await? ;
/// let d2 = df.clone();
/// let df = df.union(d2)?;
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 1 | 2 | 3 |",
/// "| 1 | 2 | 3 |",
/// "+---+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn union(self, dataframe: DataFrame) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.union(dataframe.plan)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Calculate the distinct union of two [`DataFrame`]s.
///
/// The two [`DataFrame`]s must have exactly the same schema. Any duplicate
/// rows are discarded.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let d2 = df.clone();
/// let df = df.union_distinct(d2)?;
/// // df2 are duplicate of df
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 1 | 2 | 3 |",
/// "+---+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn union_distinct(self, dataframe: DataFrame) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.union_distinct(dataframe.plan)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new `DataFrame` with all duplicated rows removed.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.distinct()?;
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 1 | 2 | 3 |",
/// "+---+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn distinct(self) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan).distinct()?.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new `DataFrame` with duplicated rows removed as per the specified expression list
/// according to the provided sorting expressions grouped by the `DISTINCT ON` clause
/// expressions.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?
/// // Return a single row (a, b) for each distinct value of a
/// .distinct_on(vec![col("a")], vec![col("a"), col("b")], None)?;
/// let expected = vec![
/// "+---+---+",
/// "| a | b |",
/// "+---+---+",
/// "| 1 | 2 |",
/// "+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn distinct_on(
self,
on_expr: Vec<Expr>,
select_expr: Vec<Expr>,
sort_expr: Option<Vec<SortExpr>>,
) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.distinct_on(on_expr, select_expr, sort_expr)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a new `DataFrame` that has statistics for a DataFrame.
///
/// Only summarizes numeric datatypes at the moment and returns nulls for
/// non numeric datatypes. The output format is modeled after pandas
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use arrow::util::pretty;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/tpch-csv/customer.csv", CsvReadOptions::new()).await?;
/// let stat = df.describe().await?;
/// # // some output column are ignored
/// let expected = vec![
/// "+------------+--------------------+--------------------+------------------------------------+--------------------+-----------------+--------------------+--------------+----------------------------------------------------------------------------------------------------------+",
/// "| describe | c_custkey | c_name | c_address | c_nationkey | c_phone | c_acctbal | c_mktsegment | c_comment |",
/// "+------------+--------------------+--------------------+------------------------------------+--------------------+-----------------+--------------------+--------------+----------------------------------------------------------------------------------------------------------+",
/// "| count | 9.0 | 9 | 9 | 9.0 | 9 | 9.0 | 9 | 9 |",
/// "| max | 10.0 | Customer#000000010 | xKiAFTjUsCuxfeleNqefumTrjS | 20.0 | 30-114-968-4951 | 9561.95 | MACHINERY | tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious |",
/// "| mean | 6.0 | null | null | 9.88888888888889 | null | 5153.2155555555555 | null | null |",
/// "| median | 6.0 | null | null | 8.0 | null | 6819.74 | null | null |",
/// "| min | 2.0 | Customer#000000002 | 6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2 | 1.0 | 11-719-748-3364 | 121.65 | AUTOMOBILE | deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov |",
/// "| null_count | 0.0 | 0 | 0 | 0.0 | 0 | 0.0 | 0 | 0 |",
/// "| std | 2.7386127875258306 | null | null | 7.2188026092359046 | null | 3522.169804254585 | null | null |",
/// "+------------+--------------------+--------------------+------------------------------------+--------------------+-----------------+--------------------+--------------+----------------------------------------------------------------------------------------------------------+"];
/// assert_batches_sorted_eq!(expected, &stat.collect().await?);
/// # Ok(())
/// # }
/// ```
pub async fn describe(self) -> Result<Self> {
//the functions now supported
let supported_describe_functions =
vec!["count", "null_count", "mean", "std", "min", "max", "median"];
let original_schema_fields = self.schema().fields().iter();
//define describe column
let mut describe_schemas = vec![Field::new("describe", DataType::Utf8, false)];
describe_schemas.extend(original_schema_fields.clone().map(|field| {
if field.data_type().is_numeric() {
Field::new(field.name(), DataType::Float64, true)
} else {
Field::new(field.name(), DataType::Utf8, true)
}
}));
//collect recordBatch
let describe_record_batch = vec![
// count aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.map(|f| count(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// null_count aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.map(|f| {
sum(case(is_null(col(f.name())))
.when(lit(true), lit(1))
.otherwise(lit(0))
.unwrap())
.alias(f.name())
})
.collect::<Vec<_>>(),
),
// mean aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| f.data_type().is_numeric())
.map(|f| avg(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// std aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| f.data_type().is_numeric())
.map(|f| stddev(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// min aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| {
!matches!(f.data_type(), DataType::Binary | DataType::Boolean)
})
.map(|f| min(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// max aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| {
!matches!(f.data_type(), DataType::Binary | DataType::Boolean)
})
.map(|f| max(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
// median aggregation
self.clone().aggregate(
vec![],
original_schema_fields
.clone()
.filter(|f| f.data_type().is_numeric())
.map(|f| median(col(f.name())).alias(f.name()))
.collect::<Vec<_>>(),
),
];
// first column with function names
let mut array_ref_vec: Vec<ArrayRef> = vec![Arc::new(StringArray::from(
supported_describe_functions.clone(),
))];
for field in original_schema_fields {
let mut array_datas = vec![];
for result in describe_record_batch.iter() {
let array_ref = match result {
Ok(df) => {
let batches = df.clone().collect().await;
match batches {
Ok(batches)
if batches.len() == 1
&& batches[0]
.column_by_name(field.name())
.is_some() =>
{
let column =
batches[0].column_by_name(field.name()).unwrap();
if column.data_type().is_null() {
Arc::new(StringArray::from(vec!["null"]))
} else if field.data_type().is_numeric() {
cast(column, &DataType::Float64)?
} else {
cast(column, &DataType::Utf8)?
}
}
_ => Arc::new(StringArray::from(vec!["null"])),
}
}
//Handling error when only boolean/binary column, and in other cases
Err(err)
if err.to_string().contains(
"Error during planning: \
Aggregate requires at least one grouping \
or aggregate expression",
) =>
{
Arc::new(StringArray::from(vec!["null"]))
}
Err(e) => return exec_err!("{}", e),
};
array_datas.push(array_ref);
}
array_ref_vec.push(concat(
array_datas
.iter()
.map(|af| af.as_ref())
.collect::<Vec<_>>()
.as_slice(),
)?);
}
let describe_record_batch =
RecordBatch::try_new(Arc::new(Schema::new(describe_schemas)), array_ref_vec)?;
let provider = MemTable::try_new(
describe_record_batch.schema(),
vec![vec![describe_record_batch]],
)?;
let plan = LogicalPlanBuilder::scan(
UNNAMED_TABLE,
provider_as_source(Arc::new(provider)),
None,
)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Apply a sort by provided expressions with default direction
pub fn sort_by(self, expr: Vec<Expr>) -> Result<DataFrame> {
self.sort(
expr.into_iter()
.map(|e| e.sort(true, false))
.collect::<Vec<SortExpr>>(),
)
}
/// Sort the DataFrame by the specified sorting expressions.
///
/// Note that any expression can be turned into
/// a sort expression by calling its [sort](Expr::sort) method.
///
/// # Example
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?;
/// let df = df.sort(vec![
/// col("a").sort(false, true), // a DESC, nulls first
/// col("b").sort(true, false), // b ASC, nulls last
/// ])?;
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 1 | 2 | 3 |",
/// "| 4 | 5 | 6 |",
/// "| 7 | 8 | 9 |",
/// "+---+---+---+",
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn sort(self, expr: Vec<SortExpr>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan).sort(expr)?.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Join this `DataFrame` with another `DataFrame` using explicitly specified
/// columns and an optional filter expression.
///
/// See [`join_on`](Self::join_on) for a more concise way to specify the
/// join condition. Since DataFusion will automatically identify and
/// optimize equality predicates there is no performance difference between
/// this function and `join_on`
///
/// `left_cols` and `right_cols` are used to form "equijoin" predicates (see
/// example below), which are then combined with the optional `filter`
/// expression.
///
/// Note that in case of outer join, the `filter` is applied to only matched rows.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let left = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let right = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?
/// .select(vec![
/// col("a").alias("a2"),
/// col("b").alias("b2"),
/// col("c").alias("c2")])?;
/// // Perform the equivalent of `left INNER JOIN right ON (a = a2 AND b = b2)`
/// // finding all pairs of rows from `left` and `right` where `a = a2` and `b = b2`.
/// let join = left.join(right, JoinType::Inner, &["a", "b"], &["a2", "b2"], None)?;
/// let expected = vec![
/// "+---+---+---+----+----+----+",
/// "| a | b | c | a2 | b2 | c2 |",
/// "+---+---+---+----+----+----+",
/// "| 1 | 2 | 3 | 1 | 2 | 3 |",
/// "+---+---+---+----+----+----+"
/// ];
/// assert_batches_sorted_eq!(expected, &join.collect().await?);
/// # Ok(())
/// # }
/// ```
///
pub fn join(
self,
right: DataFrame,
join_type: JoinType,
left_cols: &[&str],
right_cols: &[&str],
filter: Option<Expr>,
) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.join(
right.plan,
join_type,
(left_cols.to_vec(), right_cols.to_vec()),
filter,
)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Join this `DataFrame` with another `DataFrame` using the specified
/// expressions.
///
/// Note that DataFusion automatically optimizes joins, including
/// identifying and optimizing equality predicates.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let left = ctx
/// .read_csv("tests/data/example.csv", CsvReadOptions::new())
/// .await?;
/// let right = ctx
/// .read_csv("tests/data/example.csv", CsvReadOptions::new())
/// .await?
/// .select(vec![
/// col("a").alias("a2"),
/// col("b").alias("b2"),
/// col("c").alias("c2"),
/// ])?;
///
/// // Perform the equivalent of `left INNER JOIN right ON (a != a2 AND b != b2)`
/// // finding all pairs of rows from `left` and `right` where
/// // where `a != a2` and `b != b2`.
/// let join_on = left.join_on(
/// right,
/// JoinType::Inner,
/// [col("a").not_eq(col("a2")), col("b").not_eq(col("b2"))],
/// )?;
/// let expected = vec![
/// "+---+---+---+----+----+----+",
/// "| a | b | c | a2 | b2 | c2 |",
/// "+---+---+---+----+----+----+",
/// "+---+---+---+----+----+----+"
/// ];
/// # assert_batches_sorted_eq!(expected, &join_on.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn join_on(
self,
right: DataFrame,
join_type: JoinType,
on_exprs: impl IntoIterator<Item = Expr>,
) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.join_on(right.plan, join_type, on_exprs)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Repartition a DataFrame based on a logical partitioning scheme.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?;
/// let df1 = df.repartition(Partitioning::RoundRobinBatch(4))?;
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 1 | 2 | 3 |",
/// "| 4 | 5 | 6 |",
/// "| 7 | 8 | 9 |",
/// "+---+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df1.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn repartition(self, partitioning_scheme: Partitioning) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.repartition(partitioning_scheme)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return the total number of rows in this `DataFrame`.
///
/// Note that this method will actually run a plan to calculate the count,
/// which may be slow for large or complicated DataFrames.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let count = df.count().await?; // 1
/// # assert_eq!(count, 1);
/// # Ok(())
/// # }
/// ```
pub async fn count(self) -> Result<usize> {
let rows = self
.aggregate(vec![], vec![count(Expr::Literal(COUNT_STAR_EXPANSION))])?
.collect()
.await?;
let len = *rows
.first()
.and_then(|r| r.columns().first())
.and_then(|c| c.as_any().downcast_ref::<Int64Array>())
.and_then(|a| a.values().first())
.ok_or(DataFusionError::Internal(
"Unexpected output when collecting for count()".to_string(),
))? as usize;
Ok(len)
}
/// Execute this `DataFrame` and buffer all resulting `RecordBatch`es into memory.
///
/// Prior to calling `collect`, modifying a DataFrame simply updates a plan
/// (no actual computation is performed). `collect` triggers the computation.
///
/// See [`Self::execute_stream`] to execute a DataFrame without buffering.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let batches = df.collect().await?;
/// # Ok(())
/// # }
/// ```
pub async fn collect(self) -> Result<Vec<RecordBatch>> {
let task_ctx = Arc::new(self.task_ctx());
let plan = self.create_physical_plan().await?;
collect(plan, task_ctx).await
}
/// Execute the `DataFrame` and print the results to the console.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// df.show().await?;
/// # Ok(())
/// # }
/// ```
pub async fn show(self) -> Result<()> {
let results = self.collect().await?;
Ok(pretty::print_batches(&results)?)
}
/// Execute the `DataFrame` and print only the first `num` rows of the
/// result to the console.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// df.show_limit(10).await?;
/// # Ok(())
/// # }
/// ```
pub async fn show_limit(self, num: usize) -> Result<()> {
let results = self.limit(0, Some(num))?.collect().await?;
Ok(pretty::print_batches(&results)?)
}
/// Return a new [`TaskContext`] which would be used to execute this DataFrame
pub fn task_ctx(&self) -> TaskContext {
TaskContext::from(self.session_state.as_ref())
}
/// Executes this DataFrame and returns a stream over a single partition
///
/// See [Self::collect] to buffer the `RecordBatch`es in memory.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let stream = df.execute_stream().await?;
/// # Ok(())
/// # }
/// ```
///
/// # Aborting Execution
///
/// Dropping the stream will abort the execution of the query, and free up
/// any allocated resources
pub async fn execute_stream(self) -> Result<SendableRecordBatchStream> {
let task_ctx = Arc::new(self.task_ctx());
let plan = self.create_physical_plan().await?;
execute_stream(plan, task_ctx)
}
/// Executes this DataFrame and collects all results into a vector of vector of RecordBatch
/// maintaining the input partitioning.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let batches = df.collect_partitioned().await?;
/// # Ok(())
/// # }
/// ```
pub async fn collect_partitioned(self) -> Result<Vec<Vec<RecordBatch>>> {
let task_ctx = Arc::new(self.task_ctx());
let plan = self.create_physical_plan().await?;
collect_partitioned(plan, task_ctx).await
}
/// Executes this DataFrame and returns one stream per partition.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let batches = df.execute_stream_partitioned().await?;
/// # Ok(())
/// # }
/// ```
/// # Aborting Execution
///
/// Dropping the stream will abort the execution of the query, and free up
/// any allocated resources
pub async fn execute_stream_partitioned(
self,
) -> Result<Vec<SendableRecordBatchStream>> {
let task_ctx = Arc::new(self.task_ctx());
let plan = self.create_physical_plan().await?;
execute_stream_partitioned(plan, task_ctx)
}
/// Returns the `DFSchema` describing the output of this DataFrame.
///
/// The output `DFSchema` contains information on the name, data type, and
/// nullability for each column.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let schema = df.schema();
/// # Ok(())
/// # }
/// ```
pub fn schema(&self) -> &DFSchema {
self.plan.schema()
}
/// Return a reference to the unoptimized [`LogicalPlan`] that comprises
/// this DataFrame.
///
/// See [`Self::into_unoptimized_plan`] for more details.
pub fn logical_plan(&self) -> &LogicalPlan {
&self.plan
}
/// Returns both the [`LogicalPlan`] and [`SessionState`] that comprise this [`DataFrame`]
pub fn into_parts(self) -> (SessionState, LogicalPlan) {
(*self.session_state, self.plan)
}
/// Return the [`LogicalPlan`] represented by this DataFrame without running
/// any optimizers
///
/// Note: This method should not be used outside testing, as it loses the
/// snapshot of the [`SessionState`] attached to this [`DataFrame`] and
/// consequently subsequent operations may take place against a different
/// state (e.g. a different value of `now()`)
///
/// See [`Self::into_parts`] to retrieve the owned [`LogicalPlan`] and
/// corresponding [`SessionState`].
pub fn into_unoptimized_plan(self) -> LogicalPlan {
self.plan
}
/// Return the optimized [`LogicalPlan`] represented by this DataFrame.
///
/// Note: This method should not be used outside testing -- see
/// [`Self::into_unoptimized_plan`] for more details.
pub fn into_optimized_plan(self) -> Result<LogicalPlan> {
// Optimize the plan first for better UX
self.session_state.optimize(&self.plan)
}
/// Converts this [`DataFrame`] into a [`TableProvider`] that can be registered
/// as a table view using [`SessionContext::register_table`].
///
/// Note: This discards the [`SessionState`] associated with this
/// [`DataFrame`] in favour of the one passed to [`TableProvider::scan`]
pub fn into_view(self) -> Arc<dyn TableProvider> {
Arc::new(DataFrameTableProvider { plan: self.plan })
}
/// Return a DataFrame with the explanation of its plan so far.
///
/// if `analyze` is specified, runs the plan and reports metrics
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let batches = df.limit(0, Some(100))?.explain(false, false)?.collect().await?;
/// # Ok(())
/// # }
/// ```
pub fn explain(self, verbose: bool, analyze: bool) -> Result<DataFrame> {
if matches!(self.plan, LogicalPlan::Explain(_)) {
return plan_err!("Nested EXPLAINs are not supported");
}
let plan = LogicalPlanBuilder::from(self.plan)
.explain(verbose, analyze)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Return a `FunctionRegistry` used to plan udf's calls
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let f = df.registry();
/// // use f.udf("name", vec![...]) to use the udf
/// # Ok(())
/// # }
/// ```
pub fn registry(&self) -> &dyn FunctionRegistry {
self.session_state.as_ref()
}
/// Calculate the intersection of two [`DataFrame`]s. The two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let d2 = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?;
/// let df = df.intersect(d2)?;
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 1 | 2 | 3 |",
/// "+---+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &df.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn intersect(self, dataframe: DataFrame) -> Result<DataFrame> {
let left_plan = self.plan;
let right_plan = dataframe.plan;
let plan = LogicalPlanBuilder::intersect(left_plan, right_plan, true)?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Calculate the exception of two [`DataFrame`]s. The two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::assert_batches_sorted_eq;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?;
/// let d2 = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let result = df.except(d2)?;
/// // those columns are not in example.csv, but in example_long.csv
/// let expected = vec![
/// "+---+---+---+",
/// "| a | b | c |",
/// "+---+---+---+",
/// "| 4 | 5 | 6 |",
/// "| 7 | 8 | 9 |",
/// "+---+---+---+"
/// ];
/// # assert_batches_sorted_eq!(expected, &result.collect().await?);
/// # Ok(())
/// # }
/// ```
pub fn except(self, dataframe: DataFrame) -> Result<DataFrame> {
let left_plan = self.plan;
let right_plan = dataframe.plan;
let plan = LogicalPlanBuilder::except(left_plan, right_plan, true)?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Execute this `DataFrame` and write the results to `table_name`.
///
/// Returns a single [RecordBatch] containing a single column and
/// row representing the count of total rows written.
///
/// Unlike most other `DataFrame` methods, this method executes eagerly.
/// Data is written to the table using the [`TableProvider::insert_into`]
/// method. This is the same underlying implementation used by SQL `INSERT
/// INTO` statements.
pub async fn write_table(
self,
table_name: &str,
write_options: DataFrameWriteOptions,
) -> Result<Vec<RecordBatch>, DataFusionError> {
let arrow_schema = Schema::from(self.schema());
let plan = if write_options.sort_by.is_empty() {
self.plan
} else {
LogicalPlanBuilder::from(self.plan)
.sort(write_options.sort_by)?
.build()?
};
let plan = LogicalPlanBuilder::insert_into(
plan,
table_name.to_owned(),
&arrow_schema,
write_options.insert_op,
)?
.build()?;
DataFrame {
session_state: self.session_state,
plan,
}
.collect()
.await
}
/// Execute the `DataFrame` and write the results to CSV file(s).
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use std::fs;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// use datafusion::dataframe::DataFrameWriteOptions;
/// let ctx = SessionContext::new();
/// // Sort the data by column "b" and write it to a new location
/// ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?
/// .sort(vec![col("b").sort(true, true)])? // sort by b asc, nulls first
/// .write_csv(
/// "output.csv",
/// DataFrameWriteOptions::new(),
/// None, // can also specify CSV writing options here
/// ).await?;
/// # fs::remove_file("output.csv")?;
/// # Ok(())
/// # }
/// ```
pub async fn write_csv(
self,
path: &str,
options: DataFrameWriteOptions,
writer_options: Option<CsvOptions>,
) -> Result<Vec<RecordBatch>, DataFusionError> {
if options.insert_op != InsertOp::Append {
return not_impl_err!(
"{} is not implemented for DataFrame::write_csv.",
options.insert_op
);
}
let format = if let Some(csv_opts) = writer_options {
Arc::new(CsvFormatFactory::new_with_options(csv_opts))
} else {
Arc::new(CsvFormatFactory::new())
};
let file_type = format_as_file_type(format);
let plan = if options.sort_by.is_empty() {
self.plan
} else {
LogicalPlanBuilder::from(self.plan)
.sort(options.sort_by)?
.build()?
};
let plan = LogicalPlanBuilder::copy_to(
plan,
path.into(),
file_type,
HashMap::new(),
options.partition_by,
)?
.build()?;
DataFrame {
session_state: self.session_state,
plan,
}
.collect()
.await
}
/// Execute the `DataFrame` and write the results to JSON file(s).
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use std::fs;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// use datafusion::dataframe::DataFrameWriteOptions;
/// let ctx = SessionContext::new();
/// // Sort the data by column "b" and write it to a new location
/// ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?
/// .sort(vec![col("b").sort(true, true)])? // sort by b asc, nulls first
/// .write_json(
/// "output.json",
/// DataFrameWriteOptions::new(),
/// None
/// ).await?;
/// # fs::remove_file("output.json")?;
/// # Ok(())
/// # }
/// ```
pub async fn write_json(
self,
path: &str,
options: DataFrameWriteOptions,
writer_options: Option<JsonOptions>,
) -> Result<Vec<RecordBatch>, DataFusionError> {
if options.insert_op != InsertOp::Append {
return not_impl_err!(
"{} is not implemented for DataFrame::write_json.",
options.insert_op
);
}
let format = if let Some(json_opts) = writer_options {
Arc::new(JsonFormatFactory::new_with_options(json_opts))
} else {
Arc::new(JsonFormatFactory::new())
};
let file_type = format_as_file_type(format);
let plan = if options.sort_by.is_empty() {
self.plan
} else {
LogicalPlanBuilder::from(self.plan)
.sort(options.sort_by)?
.build()?
};
let plan = LogicalPlanBuilder::copy_to(
plan,
path.into(),
file_type,
Default::default(),
options.partition_by,
)?
.build()?;
DataFrame {
session_state: self.session_state,
plan,
}
.collect()
.await
}
/// Add an additional column to the DataFrame.
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.with_column("ab_sum", col("a") + col("b"))?;
/// # Ok(())
/// # }
/// ```
pub fn with_column(self, name: &str, expr: Expr) -> Result<DataFrame> {
let window_func_exprs = find_window_exprs(std::slice::from_ref(&expr));
let (window_fn_str, plan) = if window_func_exprs.is_empty() {
(None, self.plan)
} else {
(
Some(window_func_exprs[0].to_string()),
LogicalPlanBuilder::window_plan(self.plan, window_func_exprs)?,
)
};
let mut col_exists = false;
let new_column = expr.alias(name);
let mut fields: Vec<Expr> = plan
.schema()
.iter()
.filter_map(|(qualifier, field)| {
if field.name() == name {
col_exists = true;
Some(new_column.clone())
} else {
let e = col(Column::from((qualifier, field)));
window_fn_str
.as_ref()
.filter(|s| *s == &e.to_string())
.is_none()
.then_some(e)
}
})
.collect();
if !col_exists {
fields.push(new_column);
}
let project_plan = LogicalPlanBuilder::from(plan).project(fields)?.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan: project_plan,
})
}
/// Rename one column by applying a new projection. This is a no-op if the column to be
/// renamed does not exist.
///
/// The method supports case sensitive rename with wrapping column name into one of following symbols ( " or ' or ` )
///
/// Alternatively setting DataFusion param `datafusion.sql_parser.enable_ident_normalization` to `false` will enable
/// case sensitive rename without need to wrap column name into special symbols
///
/// # Example
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.with_column_renamed("ab_sum", "total")?;
///
/// # Ok(())
/// # }
/// ```
pub fn with_column_renamed(
self,
old_name: impl Into<String>,
new_name: &str,
) -> Result<DataFrame> {
let ident_opts = self
.session_state
.config_options()
.sql_parser
.enable_ident_normalization;
let old_column: Column = if ident_opts {
Column::from_qualified_name(old_name)
} else {
Column::from_qualified_name_ignore_case(old_name)
};
let (qualifier_rename, field_rename) =
match self.plan.schema().qualified_field_from_column(&old_column) {
Ok(qualifier_and_field) => qualifier_and_field,
// no-op if field not found
Err(DataFusionError::SchemaError(
SchemaError::FieldNotFound { .. },
_,
)) => return Ok(self),
Err(err) => return Err(err),
};
let projection = self
.plan
.schema()
.iter()
.map(|(qualifier, field)| {
if qualifier.eq(&qualifier_rename) && field.as_ref() == field_rename {
col(Column::from((qualifier, field))).alias(new_name)
} else {
col(Column::from((qualifier, field)))
}
})
.collect::<Vec<_>>();
let project_plan = LogicalPlanBuilder::from(self.plan)
.project(projection)?
.build()?;
Ok(DataFrame {
session_state: self.session_state,
plan: project_plan,
})
}
/// Replace all parameters in logical plan with the specified
/// values, in preparation for execution.
///
/// # Example
///
/// ```
/// use datafusion::prelude::*;
/// # use datafusion::{error::Result, assert_batches_eq};
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// # use datafusion_common::ScalarValue;
/// let ctx = SessionContext::new();
/// # ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await?;
/// let results = ctx
/// .sql("SELECT a FROM example WHERE b = $1")
/// .await?
/// // replace $1 with value 2
/// .with_param_values(vec![
/// // value at index 0 --> $1
/// ScalarValue::from(2i64)
/// ])?
/// .collect()
/// .await?;
/// assert_batches_eq!(
/// &[
/// "+---+",
/// "| a |",
/// "+---+",
/// "| 1 |",
/// "+---+",
/// ],
/// &results
/// );
/// // Note you can also provide named parameters
/// let results = ctx
/// .sql("SELECT a FROM example WHERE b = $my_param")
/// .await?
/// // replace $my_param with value 2
/// // Note you can also use a HashMap as well
/// .with_param_values(vec![
/// ("my_param", ScalarValue::from(2i64))
/// ])?
/// .collect()
/// .await?;
/// assert_batches_eq!(
/// &[
/// "+---+",
/// "| a |",
/// "+---+",
/// "| 1 |",
/// "+---+",
/// ],
/// &results
/// );
/// # Ok(())
/// # }
/// ```
pub fn with_param_values(self, query_values: impl Into<ParamValues>) -> Result<Self> {
let plan = self.plan.with_param_values(query_values)?;
Ok(DataFrame {
session_state: self.session_state,
plan,
})
}
/// Cache DataFrame as a memory table.
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
/// let df = df.cache().await?;
/// # Ok(())
/// # }
/// ```
pub async fn cache(self) -> Result<DataFrame> {
let context = SessionContext::new_with_state((*self.session_state).clone());
// The schema is consistent with the output
let plan = self.clone().create_physical_plan().await?;
let schema = plan.schema();
let task_ctx = Arc::new(self.task_ctx());
let partitions = collect_partitioned(plan, task_ctx).await?;
let mem_table = MemTable::try_new(schema, partitions)?;
context.read_table(Arc::new(mem_table))
}
}
#[derive(Debug)]
struct DataFrameTableProvider {
plan: LogicalPlan,
}
#[async_trait]
impl TableProvider for DataFrameTableProvider {
fn as_any(&self) -> &dyn Any {
self
}
fn get_logical_plan(&self) -> Option<Cow<LogicalPlan>> {
Some(Cow::Borrowed(&self.plan))
}
fn supports_filters_pushdown(
&self,
filters: &[&Expr],
) -> Result<Vec<TableProviderFilterPushDown>> {
// A filter is added on the DataFrame when given
Ok(vec![TableProviderFilterPushDown::Exact; filters.len()])
}
fn schema(&self) -> SchemaRef {
let schema: Schema = self.plan.schema().as_ref().into();
Arc::new(schema)
}
fn table_type(&self) -> TableType {
TableType::View
}
async fn scan(
&self,
state: &dyn Session,
projection: Option<&Vec<usize>>,
filters: &[Expr],
limit: Option<usize>,
) -> Result<Arc<dyn ExecutionPlan>> {
let mut expr = LogicalPlanBuilder::from(self.plan.clone());
// Add filter when given
let filter = filters.iter().cloned().reduce(|acc, new| acc.and(new));
if let Some(filter) = filter {
expr = expr.filter(filter)?
}
if let Some(p) = projection {
expr = expr.select(p.iter().copied())?
}
// add a limit if given
if let Some(l) = limit {
expr = expr.limit(0, Some(l))?
}
let plan = expr.build()?;
state.create_physical_plan(&plan).await
}
}
#[cfg(test)]
mod tests {
use std::vec;
use super::*;
use crate::assert_batches_sorted_eq;
use crate::execution::context::SessionConfig;
use crate::physical_plan::{ColumnarValue, Partitioning, PhysicalExpr};
use crate::test_util::{register_aggregate_csv, test_table, test_table_with_name};
use crate::prelude::{CsvReadOptions, NdJsonReadOptions, ParquetReadOptions};
use arrow::array::Int32Array;
use datafusion_common::{assert_batches_eq, Constraint, Constraints, ScalarValue};
use datafusion_common_runtime::SpawnedTask;
use datafusion_expr::expr::WindowFunction;
use datafusion_expr::{
cast, create_udf, lit, ExprFunctionExt, ScalarFunctionImplementation, Volatility,
WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition,
};
use datafusion_functions_aggregate::expr_fn::{array_agg, count_distinct};
use datafusion_functions_window::expr_fn::row_number;
use datafusion_functions_window::nth_value::first_value_udwf;
use datafusion_physical_expr::expressions::Column;
use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties};
use sqlparser::ast::NullTreatment;
use tempfile::TempDir;
// Get string representation of the plan
async fn assert_physical_plan(df: &DataFrame, expected: Vec<&str>) {
let physical_plan = df
.clone()
.create_physical_plan()
.await
.expect("Error creating physical plan");
let actual = get_plan_string(&physical_plan);
assert_eq!(
expected, actual,
"\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n"
);
}
pub fn table_with_constraints() -> Arc<dyn TableProvider> {
let dual_schema = Arc::new(Schema::new(vec![
Field::new("id", DataType::Int32, false),
Field::new("name", DataType::Utf8, false),
]));
let batch = RecordBatch::try_new(
dual_schema.clone(),
vec![
Arc::new(Int32Array::from(vec![1])),
Arc::new(StringArray::from(vec!["a"])),
],
)
.unwrap();
let provider = MemTable::try_new(dual_schema, vec![vec![batch]])
.unwrap()
.with_constraints(Constraints::new_unverified(vec![Constraint::PrimaryKey(
vec![0],
)]));
Arc::new(provider)
}
async fn assert_logical_expr_schema_eq_physical_expr_schema(
df: DataFrame,
) -> Result<()> {
let logical_expr_dfschema = df.schema();
let logical_expr_schema = SchemaRef::from(logical_expr_dfschema.to_owned());
let batches = df.collect().await?;
let physical_expr_schema = batches[0].schema();
assert_eq!(logical_expr_schema, physical_expr_schema);
Ok(())
}
#[tokio::test]
async fn test_array_agg_ord_schema() -> Result<()> {
let ctx = SessionContext::new();
let create_table_query = r#"
CREATE TABLE test_table (
"double_field" DOUBLE,
"string_field" VARCHAR
) AS VALUES
(1.0, 'a'),
(2.0, 'b'),
(3.0, 'c')
"#;
ctx.sql(create_table_query).await?;
let query = r#"SELECT
array_agg("double_field" ORDER BY "string_field") as "double_field",
array_agg("string_field" ORDER BY "string_field") as "string_field"
FROM test_table"#;
let result = ctx.sql(query).await?;
assert_logical_expr_schema_eq_physical_expr_schema(result).await?;
Ok(())
}
#[tokio::test]
async fn test_coalesce_schema() -> Result<()> {
let ctx = SessionContext::new();
let query = r#"SELECT COALESCE(null, 5)"#;
let result = ctx.sql(query).await?;
assert_logical_expr_schema_eq_physical_expr_schema(result).await?;
Ok(())
}
#[tokio::test]
async fn test_coalesce_from_values_schema() -> Result<()> {
let ctx = SessionContext::new();
let query = r#"SELECT COALESCE(column1, column2) FROM VALUES (null, 1.2)"#;
let result = ctx.sql(query).await?;
assert_logical_expr_schema_eq_physical_expr_schema(result).await?;
Ok(())
}
#[tokio::test]
async fn test_coalesce_from_values_schema_multiple_rows() -> Result<()> {
let ctx = SessionContext::new();
let query = r#"SELECT COALESCE(column1, column2)
FROM VALUES
(null, 1.2),
(1.1, null),
(2, 5);"#;
let result = ctx.sql(query).await?;
assert_logical_expr_schema_eq_physical_expr_schema(result).await?;
Ok(())
}
#[tokio::test]
async fn test_array_agg_schema() -> Result<()> {
let ctx = SessionContext::new();
let create_table_query = r#"
CREATE TABLE test_table (
"double_field" DOUBLE,
"string_field" VARCHAR
) AS VALUES
(1.0, 'a'),
(2.0, 'b'),
(3.0, 'c')
"#;
ctx.sql(create_table_query).await?;
let query = r#"SELECT
array_agg("double_field") as "double_field",
array_agg("string_field") as "string_field"
FROM test_table"#;
let result = ctx.sql(query).await?;
assert_logical_expr_schema_eq_physical_expr_schema(result).await?;
Ok(())
}
#[tokio::test]
async fn test_array_agg_distinct_schema() -> Result<()> {
let ctx = SessionContext::new();
let create_table_query = r#"
CREATE TABLE test_table (
"double_field" DOUBLE,
"string_field" VARCHAR
) AS VALUES
(1.0, 'a'),
(2.0, 'b'),
(2.0, 'a')
"#;
ctx.sql(create_table_query).await?;
let query = r#"SELECT
array_agg(distinct "double_field") as "double_field",
array_agg(distinct "string_field") as "string_field"
FROM test_table"#;
let result = ctx.sql(query).await?;
assert_logical_expr_schema_eq_physical_expr_schema(result).await?;
Ok(())
}
#[tokio::test]
async fn select_columns() -> Result<()> {
// build plan using Table API
let t = test_table().await?;
let t2 = t.select_columns(&["c1", "c2", "c11"])?;
let plan = t2.plan.clone();
// build query using SQL
let sql_plan = create_plan("SELECT c1, c2, c11 FROM aggregate_test_100").await?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn select_expr() -> Result<()> {
// build plan using Table API
let t = test_table().await?;
let t2 = t.select(vec![col("c1"), col("c2"), col("c11")])?;
let plan = t2.plan.clone();
// build query using SQL
let sql_plan = create_plan("SELECT c1, c2, c11 FROM aggregate_test_100").await?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn select_exprs() -> Result<()> {
// build plan using `select_expr``
let t = test_table().await?;
let plan = t
.clone()
.select_exprs(&["c1", "c2", "c11", "c2 * c11"])?
.plan;
// build plan using select
let expected_plan = t
.select(vec![
col("c1"),
col("c2"),
col("c11"),
col("c2") * col("c11"),
])?
.plan;
assert_same_plan(&expected_plan, &plan);
Ok(())
}
#[tokio::test]
async fn select_with_window_exprs() -> Result<()> {
// build plan using Table API
let t = test_table().await?;
let first_row = Expr::WindowFunction(WindowFunction::new(
WindowFunctionDefinition::WindowUDF(first_value_udwf()),
vec![col("aggregate_test_100.c1")],
))
.partition_by(vec![col("aggregate_test_100.c2")])
.build()
.unwrap();
let t2 = t.select(vec![col("c1"), first_row])?;
let plan = t2.plan.clone();
let sql_plan = create_plan(
"select c1, first_value(c1) over (partition by c2) from aggregate_test_100",
)
.await?;
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn select_with_periods() -> Result<()> {
// define data with a column name that has a "." in it:
let array: Int32Array = [1, 10].into_iter().collect();
let batch = RecordBatch::try_from_iter(vec![("f.c1", Arc::new(array) as _)])?;
let ctx = SessionContext::new();
ctx.register_batch("t", batch)?;
let df = ctx.table("t").await?.select_columns(&["f.c1"])?;
let df_results = df.collect().await?;
assert_batches_sorted_eq!(
["+------+", "| f.c1 |", "+------+", "| 1 |", "| 10 |", "+------+"],
&df_results
);
Ok(())
}
#[tokio::test]
async fn drop_columns() -> Result<()> {
// build plan using Table API
let t = test_table().await?;
let t2 = t.drop_columns(&["c2", "c11"])?;
let plan = t2.plan.clone();
// build query using SQL
let sql_plan = create_plan(
"SELECT c1,c3,c4,c5,c6,c7,c8,c9,c10,c12,c13 FROM aggregate_test_100",
)
.await?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn drop_columns_with_duplicates() -> Result<()> {
// build plan using Table API
let t = test_table().await?;
let t2 = t.drop_columns(&["c2", "c11", "c2", "c2"])?;
let plan = t2.plan.clone();
// build query using SQL
let sql_plan = create_plan(
"SELECT c1,c3,c4,c5,c6,c7,c8,c9,c10,c12,c13 FROM aggregate_test_100",
)
.await?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn drop_columns_with_nonexistent_columns() -> Result<()> {
// build plan using Table API
let t = test_table().await?;
let t2 = t.drop_columns(&["canada", "c2", "rocks"])?;
let plan = t2.plan.clone();
// build query using SQL
let sql_plan = create_plan(
"SELECT c1,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13 FROM aggregate_test_100",
)
.await?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn drop_columns_with_empty_array() -> Result<()> {
// build plan using Table API
let t = test_table().await?;
let t2 = t.drop_columns(&[])?;
let plan = t2.plan.clone();
// build query using SQL
let sql_plan = create_plan(
"SELECT c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13 FROM aggregate_test_100",
)
.await?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn drop_with_quotes() -> Result<()> {
// define data with a column name that has a "." in it:
let array1: Int32Array = [1, 10].into_iter().collect();
let array2: Int32Array = [2, 11].into_iter().collect();
let batch = RecordBatch::try_from_iter(vec![
("f\"c1", Arc::new(array1) as _),
("f\"c2", Arc::new(array2) as _),
])?;
let ctx = SessionContext::new();
ctx.register_batch("t", batch)?;
let df = ctx.table("t").await?.drop_columns(&["f\"c1"])?;
let df_results = df.collect().await?;
assert_batches_sorted_eq!(
[
"+------+",
"| f\"c2 |",
"+------+",
"| 2 |",
"| 11 |",
"+------+"
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn drop_with_periods() -> Result<()> {
// define data with a column name that has a "." in it:
let array1: Int32Array = [1, 10].into_iter().collect();
let array2: Int32Array = [2, 11].into_iter().collect();
let batch = RecordBatch::try_from_iter(vec![
("f.c1", Arc::new(array1) as _),
("f.c2", Arc::new(array2) as _),
])?;
let ctx = SessionContext::new();
ctx.register_batch("t", batch)?;
let df = ctx.table("t").await?.drop_columns(&["f.c1"])?;
let df_results = df.collect().await?;
assert_batches_sorted_eq!(
["+------+", "| f.c2 |", "+------+", "| 2 |", "| 11 |", "+------+"],
&df_results
);
Ok(())
}
#[tokio::test]
async fn aggregate() -> Result<()> {
// build plan using DataFrame API
let df = test_table().await?;
let group_expr = vec![col("c1")];
let aggr_expr = vec![
min(col("c12")),
max(col("c12")),
avg(col("c12")),
sum(col("c12")),
count(col("c12")),
count_distinct(col("c12")),
];
let df: Vec<RecordBatch> = df.aggregate(group_expr, aggr_expr)?.collect().await?;
assert_batches_sorted_eq!(
["+----+-----------------------------+-----------------------------+-----------------------------+-----------------------------+-------------------------------+----------------------------------------+",
"| c1 | min(aggregate_test_100.c12) | max(aggregate_test_100.c12) | avg(aggregate_test_100.c12) | sum(aggregate_test_100.c12) | count(aggregate_test_100.c12) | count(DISTINCT aggregate_test_100.c12) |",
"+----+-----------------------------+-----------------------------+-----------------------------+-----------------------------+-------------------------------+----------------------------------------+",
"| a | 0.02182578039211991 | 0.9800193410444061 | 0.48754517466109415 | 10.238448667882977 | 21 | 21 |",
"| b | 0.04893135681998029 | 0.9185813970744787 | 0.41040709263815384 | 7.797734760124923 | 19 | 19 |",
"| c | 0.0494924465469434 | 0.991517828651004 | 0.6600456536439784 | 13.860958726523545 | 21 | 21 |",
"| d | 0.061029375346466685 | 0.9748360509016578 | 0.48855379387549824 | 8.793968289758968 | 18 | 18 |",
"| e | 0.01479305307777301 | 0.9965400387585364 | 0.48600669271341534 | 10.206140546981722 | 21 | 21 |",
"+----+-----------------------------+-----------------------------+-----------------------------+-----------------------------+-------------------------------+----------------------------------------+"],
&df
);
Ok(())
}
#[tokio::test]
async fn aggregate_assert_no_empty_batches() -> Result<()> {
// build plan using DataFrame API
let df = test_table().await?;
let group_expr = vec![col("c1")];
let aggr_expr = vec![
min(col("c12")),
max(col("c12")),
avg(col("c12")),
sum(col("c12")),
count(col("c12")),
count_distinct(col("c12")),
median(col("c12")),
];
let df: Vec<RecordBatch> = df.aggregate(group_expr, aggr_expr)?.collect().await?;
// Empty batches should not be produced
for batch in df {
assert!(batch.num_rows() > 0);
}
Ok(())
}
#[tokio::test]
async fn test_aggregate_with_pk() -> Result<()> {
// create the dataframe
let config = SessionConfig::new().with_target_partitions(1);
let ctx = SessionContext::new_with_config(config);
let df = ctx.read_table(table_with_constraints())?;
// GROUP BY id
let group_expr = vec![col("id")];
let aggr_expr = vec![];
let df = df.aggregate(group_expr, aggr_expr)?;
// Since id and name are functionally dependant, we can use name among
// expression even if it is not part of the group by expression and can
// select "name" column even though it wasn't explicitly grouped
let df = df.select(vec![col("id"), col("name")])?;
assert_physical_plan(
&df,
vec![
"AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]",
" MemoryExec: partitions=1, partition_sizes=[1]",
],
)
.await;
let df_results = df.collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!([
"+----+------+",
"| id | name |",
"+----+------+",
"| 1 | a |",
"+----+------+"
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_aggregate_with_pk2() -> Result<()> {
// create the dataframe
let config = SessionConfig::new().with_target_partitions(1);
let ctx = SessionContext::new_with_config(config);
let df = ctx.read_table(table_with_constraints())?;
// GROUP BY id
let group_expr = vec![col("id")];
let aggr_expr = vec![];
let df = df.aggregate(group_expr, aggr_expr)?;
// Predicate refers to id, and name fields:
// id = 1 AND name = 'a'
let predicate = col("id").eq(lit(1i32)).and(col("name").eq(lit("a")));
let df = df.filter(predicate)?;
assert_physical_plan(
&df,
vec![
"CoalesceBatchesExec: target_batch_size=8192",
" FilterExec: id@0 = 1 AND name@1 = a",
" AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]",
" MemoryExec: partitions=1, partition_sizes=[1]",
],
)
.await;
// Since id and name are functionally dependant, we can use name among expression
// even if it is not part of the group by expression.
let df_results = df.collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!(
["+----+------+",
"| id | name |",
"+----+------+",
"| 1 | a |",
"+----+------+",],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_aggregate_with_pk3() -> Result<()> {
// create the dataframe
let config = SessionConfig::new().with_target_partitions(1);
let ctx = SessionContext::new_with_config(config);
let df = ctx.read_table(table_with_constraints())?;
// GROUP BY id
let group_expr = vec![col("id")];
let aggr_expr = vec![];
// group by id,
let df = df.aggregate(group_expr, aggr_expr)?;
// Predicate refers to id field
// id = 1
let predicate = col("id").eq(lit(1i32));
let df = df.filter(predicate)?;
// Select expression refers to id, and name columns.
// id, name
let df = df.select(vec![col("id"), col("name")])?;
assert_physical_plan(
&df,
vec![
"CoalesceBatchesExec: target_batch_size=8192",
" FilterExec: id@0 = 1",
" AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]",
" MemoryExec: partitions=1, partition_sizes=[1]",
],
)
.await;
// Since id and name are functionally dependant, we can use name among expression
// even if it is not part of the group by expression.
let df_results = df.collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!(
["+----+------+",
"| id | name |",
"+----+------+",
"| 1 | a |",
"+----+------+",],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_aggregate_with_pk4() -> Result<()> {
// create the dataframe
let config = SessionConfig::new().with_target_partitions(1);
let ctx = SessionContext::new_with_config(config);
let df = ctx.read_table(table_with_constraints())?;
// GROUP BY id
let group_expr = vec![col("id")];
let aggr_expr = vec![];
let df = df.aggregate(group_expr, aggr_expr)?;
// Predicate refers to id field
// id = 1
let predicate = col("id").eq(lit(1i32));
let df = df.filter(predicate)?;
// Select expression refers to id column.
// id
let df = df.select(vec![col("id")])?;
// In this case aggregate shouldn't be expanded, since these
// columns are not used.
assert_physical_plan(
&df,
vec![
"CoalesceBatchesExec: target_batch_size=8192",
" FilterExec: id@0 = 1",
" AggregateExec: mode=Single, gby=[id@0 as id], aggr=[]",
" MemoryExec: partitions=1, partition_sizes=[1]",
],
)
.await;
let df_results = df.collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!([
"+----+",
"| id |",
"+----+",
"| 1 |",
"+----+",],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_aggregate_alias() -> Result<()> {
let df = test_table().await?;
let df = df
// GROUP BY `c2 + 1`
.aggregate(vec![col("c2") + lit(1)], vec![])?
// SELECT `c2 + 1` as c2
.select(vec![(col("c2") + lit(1)).alias("c2")])?
// GROUP BY c2 as "c2" (alias in expr is not supported by SQL)
.aggregate(vec![col("c2").alias("c2")], vec![])?;
let df_results = df.collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!([
"+----+",
"| c2 |",
"+----+",
"| 2 |",
"| 3 |",
"| 4 |",
"| 5 |",
"| 6 |",
"+----+",
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_aggregate_with_union() -> Result<()> {
let df = test_table().await?;
let df1 = df
.clone()
// GROUP BY `c1`
.aggregate(vec![col("c1")], vec![min(col("c2"))])?
// SELECT `c1` , min(c2) as `result`
.select(vec![col("c1"), min(col("c2")).alias("result")])?;
let df2 = df
.clone()
// GROUP BY `c1`
.aggregate(vec![col("c1")], vec![max(col("c3"))])?
// SELECT `c1` , max(c3) as `result`
.select(vec![col("c1"), max(col("c3")).alias("result")])?;
let df_union = df1.union(df2)?;
let df = df_union
// GROUP BY `c1`
.aggregate(
vec![col("c1")],
vec![sum(col("result")).alias("sum_result")],
)?
// SELECT `c1`, sum(result) as `sum_result`
.select(vec![(col("c1")), col("sum_result")])?;
let df_results = df.collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!(
[
"+----+------------+",
"| c1 | sum_result |",
"+----+------------+",
"| a | 84 |",
"| b | 69 |",
"| c | 124 |",
"| d | 126 |",
"| e | 121 |",
"+----+------------+"
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_aggregate_subexpr() -> Result<()> {
let df = test_table().await?;
let group_expr = col("c2") + lit(1);
let aggr_expr = sum(col("c3") + lit(2));
let df = df
// GROUP BY `c2 + 1`
.aggregate(vec![group_expr.clone()], vec![aggr_expr.clone()])?
// SELECT `c2 + 1` as c2 + 10, sum(c3 + 2) + 20
// SELECT expressions contain aggr_expr and group_expr as subexpressions
.select(vec![
group_expr.alias("c2") + lit(10),
(aggr_expr + lit(20)).alias("sum"),
])?;
let df_results = df.collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!([
"+----------------+------+",
"| c2 + Int32(10) | sum |",
"+----------------+------+",
"| 12 | 431 |",
"| 13 | 248 |",
"| 14 | 453 |",
"| 15 | 95 |",
"| 16 | -146 |",
"+----------------+------+",
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_aggregate_name_collision() -> Result<()> {
let df = test_table().await?;
let collided_alias = "aggregate_test_100.c2 + aggregate_test_100.c3";
let group_expr = lit(1).alias(collided_alias);
let df = df
// GROUP BY 1
.aggregate(vec![group_expr], vec![])?
// SELECT `aggregate_test_100.c2 + aggregate_test_100.c3`
.select(vec![
(col("aggregate_test_100.c2") + col("aggregate_test_100.c3")),
])
// The select expr has the same display_name as the group_expr,
// but since they are different expressions, it should fail.
.expect_err("Expected error");
let expected = "Schema error: No field named aggregate_test_100.c2. \
Valid fields are \"aggregate_test_100.c2 + aggregate_test_100.c3\".";
assert_eq!(df.strip_backtrace(), expected);
Ok(())
}
#[tokio::test]
async fn window_using_aggregates() -> Result<()> {
// build plan using DataFrame API
let df = test_table().await?.filter(col("c1").eq(lit("a")))?;
let mut aggr_expr = vec![
(
datafusion_functions_aggregate::first_last::first_value_udaf(),
"first_value",
),
(
datafusion_functions_aggregate::first_last::last_value_udaf(),
"last_val",
),
(
datafusion_functions_aggregate::approx_distinct::approx_distinct_udaf(),
"approx_distinct",
),
(
datafusion_functions_aggregate::approx_median::approx_median_udaf(),
"approx_median",
),
(
datafusion_functions_aggregate::median::median_udaf(),
"median",
),
(datafusion_functions_aggregate::min_max::max_udaf(), "max"),
(datafusion_functions_aggregate::min_max::min_udaf(), "min"),
]
.into_iter()
.map(|(func, name)| {
let w = WindowFunction::new(
WindowFunctionDefinition::AggregateUDF(func),
vec![col("c3")],
);
Expr::WindowFunction(w)
.null_treatment(NullTreatment::IgnoreNulls)
.order_by(vec![col("c2").sort(true, true), col("c3").sort(true, true)])
.window_frame(WindowFrame::new_bounds(
WindowFrameUnits::Rows,
WindowFrameBound::Preceding(ScalarValue::UInt64(None)),
WindowFrameBound::Preceding(ScalarValue::UInt64(Some(1))),
))
.build()
.unwrap()
.alias(name)
})
.collect::<Vec<_>>();
aggr_expr.extend_from_slice(&[col("c2"), col("c3")]);
let df: Vec<RecordBatch> = df.select(aggr_expr)?.collect().await?;
assert_batches_sorted_eq!(
[
"+-------------+----------+-----------------+---------------+--------+-----+------+----+------+",
"| first_value | last_val | approx_distinct | approx_median | median | max | min | c2 | c3 |",
"+-------------+----------+-----------------+---------------+--------+-----+------+----+------+",
"| | | | | | | | 1 | -85 |",
"| -85 | -101 | 14 | -12 | -101 | 83 | -101 | 4 | -54 |",
"| -85 | -101 | 17 | -25 | -101 | 83 | -101 | 5 | -31 |",
"| -85 | -12 | 10 | -32 | -12 | 83 | -85 | 3 | 13 |",
"| -85 | -25 | 3 | -56 | -25 | -25 | -85 | 1 | -5 |",
"| -85 | -31 | 18 | -29 | -31 | 83 | -101 | 5 | 36 |",
"| -85 | -38 | 16 | -25 | -38 | 83 | -101 | 4 | 65 |",
"| -85 | -43 | 7 | -43 | -43 | 83 | -85 | 2 | 45 |",
"| -85 | -48 | 6 | -35 | -48 | 83 | -85 | 2 | -43 |",
"| -85 | -5 | 4 | -37 | -5 | -5 | -85 | 1 | 83 |",
"| -85 | -54 | 15 | -17 | -54 | 83 | -101 | 4 | -38 |",
"| -85 | -56 | 2 | -70 | -56 | -56 | -85 | 1 | -25 |",
"| -85 | -72 | 9 | -43 | -72 | 83 | -85 | 3 | -12 |",
"| -85 | -85 | 1 | -85 | -85 | -85 | -85 | 1 | -56 |",
"| -85 | 13 | 11 | -17 | 13 | 83 | -85 | 3 | 14 |",
"| -85 | 13 | 11 | -25 | 13 | 83 | -85 | 3 | 13 |",
"| -85 | 14 | 12 | -12 | 14 | 83 | -85 | 3 | 17 |",
"| -85 | 17 | 13 | -11 | 17 | 83 | -85 | 4 | -101 |",
"| -85 | 45 | 8 | -34 | 45 | 83 | -85 | 3 | -72 |",
"| -85 | 65 | 17 | -17 | 65 | 83 | -101 | 5 | -101 |",
"| -85 | 83 | 5 | -25 | 83 | 83 | -85 | 2 | -48 |",
"+-------------+----------+-----------------+---------------+--------+-----+------+----+------+",
],
&df
);
Ok(())
}
// Test issue: https://github.com/apache/datafusion/issues/10346
#[tokio::test]
async fn test_select_over_aggregate_schema() -> Result<()> {
let df = test_table()
.await?
.with_column("c", col("c1"))?
.aggregate(vec![], vec![array_agg(col("c")).alias("c")])?
.select(vec![col("c")])?;
assert_eq!(df.schema().fields().len(), 1);
let field = df.schema().field(0);
// There are two columns named 'c', one from the input of the aggregate and the other from the output.
// Select should return the column from the output of the aggregate, which is a list.
assert!(matches!(field.data_type(), DataType::List(_)));
Ok(())
}
#[tokio::test]
async fn test_distinct() -> Result<()> {
let t = test_table().await?;
let plan = t
.select(vec![col("c1")])
.unwrap()
.distinct()
.unwrap()
.plan
.clone();
let sql_plan = create_plan("select distinct c1 from aggregate_test_100").await?;
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn test_distinct_sort_by() -> Result<()> {
let t = test_table().await?;
let plan = t
.select(vec![col("c1")])
.unwrap()
.distinct()
.unwrap()
.sort(vec![col("c1").sort(true, true)])
.unwrap();
let df_results = plan.clone().collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!(
["+----+",
"| c1 |",
"+----+",
"| a |",
"| b |",
"| c |",
"| d |",
"| e |",
"+----+"],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_distinct_sort_by_unprojected() -> Result<()> {
let t = test_table().await?;
let err = t
.select(vec![col("c1")])
.unwrap()
.distinct()
.unwrap()
// try to sort on some value not present in input to distinct
.sort(vec![col("c2").sort(true, true)])
.unwrap_err();
assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions c2 must appear in select list");
Ok(())
}
#[tokio::test]
async fn test_distinct_on() -> Result<()> {
let t = test_table().await?;
let plan = t
.distinct_on(vec![col("c1")], vec![col("aggregate_test_100.c1")], None)
.unwrap();
let sql_plan =
create_plan("select distinct on (c1) c1 from aggregate_test_100").await?;
assert_same_plan(&plan.plan.clone(), &sql_plan);
let df_results = plan.clone().collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!(
["+----+",
"| c1 |",
"+----+",
"| a |",
"| b |",
"| c |",
"| d |",
"| e |",
"+----+"],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_distinct_on_sort_by() -> Result<()> {
let t = test_table().await?;
let plan = t
.select(vec![col("c1")])
.unwrap()
.distinct_on(
vec![col("c1")],
vec![col("c1")],
Some(vec![col("c1").sort(true, true)]),
)
.unwrap()
.sort(vec![col("c1").sort(true, true)])
.unwrap();
let df_results = plan.clone().collect().await?;
#[rustfmt::skip]
assert_batches_sorted_eq!(
["+----+",
"| c1 |",
"+----+",
"| a |",
"| b |",
"| c |",
"| d |",
"| e |",
"+----+"],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_distinct_on_sort_by_unprojected() -> Result<()> {
let t = test_table().await?;
let err = t
.select(vec![col("c1")])
.unwrap()
.distinct_on(
vec![col("c1")],
vec![col("c1")],
Some(vec![col("c1").sort(true, true)]),
)
.unwrap()
// try to sort on some value not present in input to distinct
.sort(vec![col("c2").sort(true, true)])
.unwrap_err();
assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions c2 must appear in select list");
Ok(())
}
#[tokio::test]
async fn join() -> Result<()> {
let left = test_table().await?.select_columns(&["c1", "c2"])?;
let right = test_table_with_name("c2")
.await?
.select_columns(&["c1", "c3"])?;
let left_rows = left.clone().collect().await?;
let right_rows = right.clone().collect().await?;
let join = left.join(right, JoinType::Inner, &["c1"], &["c1"], None)?;
let join_rows = join.collect().await?;
assert_eq!(100, left_rows.iter().map(|x| x.num_rows()).sum::<usize>());
assert_eq!(100, right_rows.iter().map(|x| x.num_rows()).sum::<usize>());
assert_eq!(2008, join_rows.iter().map(|x| x.num_rows()).sum::<usize>());
Ok(())
}
#[tokio::test]
async fn join_on() -> Result<()> {
let left = test_table_with_name("a")
.await?
.select_columns(&["c1", "c2"])?;
let right = test_table_with_name("b")
.await?
.select_columns(&["c1", "c2"])?;
let join = left.join_on(
right,
JoinType::Inner,
[col("a.c1").not_eq(col("b.c1")), col("a.c2").eq(col("b.c2"))],
)?;
let expected_plan = "Inner Join: Filter: a.c1 != b.c1 AND a.c2 = b.c2\
\n Projection: a.c1, a.c2\
\n TableScan: a\
\n Projection: b.c1, b.c2\
\n TableScan: b";
assert_eq!(expected_plan, format!("{}", join.logical_plan()));
Ok(())
}
#[tokio::test]
async fn join_on_filter_datatype() -> Result<()> {
let left = test_table_with_name("a").await?.select_columns(&["c1"])?;
let right = test_table_with_name("b").await?.select_columns(&["c1"])?;
// JOIN ON untyped NULL
let join = left.clone().join_on(
right.clone(),
JoinType::Inner,
Some(Expr::Literal(ScalarValue::Null)),
)?;
let expected_plan = "EmptyRelation";
assert_eq!(expected_plan, format!("{}", join.into_optimized_plan()?));
// JOIN ON expression must be boolean type
let join = left.join_on(right, JoinType::Inner, Some(lit("TRUE")))?;
let expected = join.into_optimized_plan().unwrap_err();
assert_eq!(
expected.strip_backtrace(),
"type_coercion\ncaused by\nError during planning: Join condition must be boolean type, but got Utf8"
);
Ok(())
}
#[tokio::test]
async fn join_ambiguous_filter() -> Result<()> {
let left = test_table_with_name("a")
.await?
.select_columns(&["c1", "c2"])?;
let right = test_table_with_name("b")
.await?
.select_columns(&["c1", "c2"])?;
let join = left
.join_on(right, JoinType::Inner, [col("c1").eq(col("c1"))])
.expect_err("join didn't fail check");
let expected = "Schema error: Ambiguous reference to unqualified field c1";
assert_eq!(join.strip_backtrace(), expected);
Ok(())
}
#[tokio::test]
async fn limit() -> Result<()> {
// build query using Table API
let t = test_table().await?;
let t2 = t.select_columns(&["c1", "c2", "c11"])?.limit(0, Some(10))?;
let plan = t2.plan.clone();
// build query using SQL
let sql_plan =
create_plan("SELECT c1, c2, c11 FROM aggregate_test_100 LIMIT 10").await?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn df_count() -> Result<()> {
let count = test_table().await?.count().await?;
assert_eq!(100, count);
Ok(())
}
#[tokio::test]
async fn explain() -> Result<()> {
// build query using Table API
let df = test_table().await?;
let df = df
.select_columns(&["c1", "c2", "c11"])?
.limit(0, Some(10))?
.explain(false, false)?;
let plan = df.plan.clone();
// build query using SQL
let sql_plan =
create_plan("EXPLAIN SELECT c1, c2, c11 FROM aggregate_test_100 LIMIT 10")
.await?;
// the two plans should be identical
assert_same_plan(&plan, &sql_plan);
Ok(())
}
#[tokio::test]
async fn registry() -> Result<()> {
let ctx = SessionContext::new();
register_aggregate_csv(&ctx, "aggregate_test_100").await?;
// declare the udf
let my_fn: ScalarFunctionImplementation =
Arc::new(|_: &[ColumnarValue]| unimplemented!("my_fn is not implemented"));
// create and register the udf
ctx.register_udf(create_udf(
"my_fn",
vec![DataType::Float64],
DataType::Float64,
Volatility::Immutable,
my_fn,
));
// build query with a UDF using DataFrame API
let df = ctx.table("aggregate_test_100").await?;
let expr = df.registry().udf("my_fn")?.call(vec![col("c12")]);
let df = df.select(vec![expr])?;
// build query using SQL
let sql_plan = ctx.sql("SELECT my_fn(c12) FROM aggregate_test_100").await?;
// the two plans should be identical
assert_same_plan(&df.plan, &sql_plan.plan);
Ok(())
}
#[tokio::test]
async fn sendable() {
let df = test_table().await.unwrap();
// dataframes should be sendable between threads/tasks
let task = SpawnedTask::spawn(async move {
df.select_columns(&["c1"])
.expect("should be usable in a task")
});
task.join().await.expect("task completed successfully");
}
#[tokio::test]
async fn intersect() -> Result<()> {
let df = test_table().await?.select_columns(&["c1", "c3"])?;
let d2 = df.clone();
let plan = df.intersect(d2)?;
let result = plan.plan.clone();
let expected = create_plan(
"SELECT c1, c3 FROM aggregate_test_100
INTERSECT ALL SELECT c1, c3 FROM aggregate_test_100",
)
.await?;
assert_same_plan(&result, &expected);
Ok(())
}
#[tokio::test]
async fn except() -> Result<()> {
let df = test_table().await?.select_columns(&["c1", "c3"])?;
let d2 = df.clone();
let plan = df.except(d2)?;
let result = plan.plan.clone();
let expected = create_plan(
"SELECT c1, c3 FROM aggregate_test_100
EXCEPT ALL SELECT c1, c3 FROM aggregate_test_100",
)
.await?;
assert_same_plan(&result, &expected);
Ok(())
}
#[tokio::test]
async fn register_table() -> Result<()> {
let df = test_table().await?.select_columns(&["c1", "c12"])?;
let ctx = SessionContext::new();
let df_impl = DataFrame::new(ctx.state(), df.plan.clone());
// register a dataframe as a table
ctx.register_table("test_table", df_impl.clone().into_view())?;
// pull the table out
let table = ctx.table("test_table").await?;
let group_expr = vec![col("c1")];
let aggr_expr = vec![sum(col("c12"))];
// check that we correctly read from the table
let df_results = df_impl
.aggregate(group_expr.clone(), aggr_expr.clone())?
.collect()
.await?;
let table_results = &table.aggregate(group_expr, aggr_expr)?.collect().await?;
assert_batches_sorted_eq!(
[
"+----+-----------------------------+",
"| c1 | sum(aggregate_test_100.c12) |",
"+----+-----------------------------+",
"| a | 10.238448667882977 |",
"| b | 7.797734760124923 |",
"| c | 13.860958726523545 |",
"| d | 8.793968289758968 |",
"| e | 10.206140546981722 |",
"+----+-----------------------------+"
],
&df_results
);
// the results are the same as the results from the view, modulo the leaf table name
assert_batches_sorted_eq!(
[
"+----+---------------------+",
"| c1 | sum(test_table.c12) |",
"+----+---------------------+",
"| a | 10.238448667882977 |",
"| b | 7.797734760124923 |",
"| c | 13.860958726523545 |",
"| d | 8.793968289758968 |",
"| e | 10.206140546981722 |",
"+----+---------------------+"
],
table_results
);
Ok(())
}
/// Compare the formatted string representation of two plans for equality
fn assert_same_plan(plan1: &LogicalPlan, plan2: &LogicalPlan) {
assert_eq!(format!("{plan1:?}"), format!("{plan2:?}"));
}
/// Create a logical plan from a SQL query
async fn create_plan(sql: &str) -> Result<LogicalPlan> {
let ctx = SessionContext::new();
register_aggregate_csv(&ctx, "aggregate_test_100").await?;
Ok(ctx.sql(sql).await?.into_unoptimized_plan())
}
#[tokio::test]
async fn with_column() -> Result<()> {
let df = test_table().await?.select_columns(&["c1", "c2", "c3"])?;
let ctx = SessionContext::new();
let df_impl = DataFrame::new(ctx.state(), df.plan.clone());
let df = df_impl
.filter(col("c2").eq(lit(3)).and(col("c1").eq(lit("a"))))?
.with_column("sum", col("c2") + col("c3"))?;
// check that new column added
let df_results = df.clone().collect().await?;
assert_batches_sorted_eq!(
[
"+----+----+-----+-----+",
"| c1 | c2 | c3 | sum |",
"+----+----+-----+-----+",
"| a | 3 | -12 | -9 |",
"| a | 3 | -72 | -69 |",
"| a | 3 | 13 | 16 |",
"| a | 3 | 13 | 16 |",
"| a | 3 | 14 | 17 |",
"| a | 3 | 17 | 20 |",
"+----+----+-----+-----+"
],
&df_results
);
// check that col with the same name overwritten
let df_results_overwrite = df
.clone()
.with_column("c1", col("c2") + col("c3"))?
.collect()
.await?;
assert_batches_sorted_eq!(
[
"+-----+----+-----+-----+",
"| c1 | c2 | c3 | sum |",
"+-----+----+-----+-----+",
"| -69 | 3 | -72 | -69 |",
"| -9 | 3 | -12 | -9 |",
"| 16 | 3 | 13 | 16 |",
"| 16 | 3 | 13 | 16 |",
"| 17 | 3 | 14 | 17 |",
"| 20 | 3 | 17 | 20 |",
"+-----+----+-----+-----+"
],
&df_results_overwrite
);
// check that col with the same name overwritten using same name as reference
let df_results_overwrite_self = df
.clone()
.with_column("c2", col("c2") + lit(1))?
.collect()
.await?;
assert_batches_sorted_eq!(
[
"+----+----+-----+-----+",
"| c1 | c2 | c3 | sum |",
"+----+----+-----+-----+",
"| a | 4 | -12 | -9 |",
"| a | 4 | -72 | -69 |",
"| a | 4 | 13 | 16 |",
"| a | 4 | 13 | 16 |",
"| a | 4 | 14 | 17 |",
"| a | 4 | 17 | 20 |",
"+----+----+-----+-----+"
],
&df_results_overwrite_self
);
Ok(())
}
// Test issues: https://github.com/apache/datafusion/issues/11982
// and https://github.com/apache/datafusion/issues/12425
// Window function was creating unwanted projection when using with_column() method.
#[tokio::test]
async fn test_window_function_with_column() -> Result<()> {
let df = test_table().await?.select_columns(&["c1", "c2", "c3"])?;
let ctx = SessionContext::new();
let df_impl = DataFrame::new(ctx.state(), df.plan.clone());
let func = row_number().alias("row_num");
// This first `with_column` results in a column without a `qualifier`
let df_impl = df_impl.with_column("s", col("c2") + col("c3"))?;
// This second `with_column` should only alias `func` as `"r"`
let df = df_impl.with_column("r", func)?.limit(0, Some(2))?;
df.clone().show().await?;
assert_eq!(5, df.schema().fields().len());
let df_results = df.clone().collect().await?;
assert_batches_sorted_eq!(
[
"+----+----+-----+-----+---+",
"| c1 | c2 | c3 | s | r |",
"+----+----+-----+-----+---+",
"| c | 2 | 1 | 3 | 1 |",
"| d | 5 | -40 | -35 | 2 |",
"+----+----+-----+-----+---+",
],
&df_results
);
Ok(())
}
// Test issue: https://github.com/apache/datafusion/issues/7790
// The join operation outputs two identical column names, but they belong to different relations.
#[tokio::test]
async fn with_column_join_same_columns() -> Result<()> {
let df = test_table().await?.select_columns(&["c1"])?;
let ctx = SessionContext::new();
let table = df.into_view();
ctx.register_table("t1", table.clone())?;
ctx.register_table("t2", table)?;
let df = ctx
.table("t1")
.await?
.join(
ctx.table("t2").await?,
JoinType::Inner,
&["c1"],
&["c1"],
None,
)?
.sort(vec![
// make the test deterministic
col("t1.c1").sort(true, true),
])?
.limit(0, Some(1))?;
let df_results = df.clone().collect().await?;
assert_batches_sorted_eq!(
[
"+----+----+",
"| c1 | c1 |",
"+----+----+",
"| a | a |",
"+----+----+",
],
&df_results
);
let df_with_column = df.clone().with_column("new_column", lit(true))?;
assert_eq!(
"\
Projection: t1.c1, t2.c1, Boolean(true) AS new_column\
\n Limit: skip=0, fetch=1\
\n Sort: t1.c1 ASC NULLS FIRST\
\n Inner Join: t1.c1 = t2.c1\
\n TableScan: t1\
\n TableScan: t2",
format!("{}", df_with_column.logical_plan())
);
assert_eq!(
"\
Projection: t1.c1, t2.c1, Boolean(true) AS new_column\
\n Sort: t1.c1 ASC NULLS FIRST, fetch=1\
\n Inner Join: t1.c1 = t2.c1\
\n SubqueryAlias: t1\
\n TableScan: aggregate_test_100 projection=[c1]\
\n SubqueryAlias: t2\
\n TableScan: aggregate_test_100 projection=[c1]",
format!("{}", df_with_column.clone().into_optimized_plan()?)
);
let df_results = df_with_column.collect().await?;
assert_batches_sorted_eq!(
[
"+----+----+------------+",
"| c1 | c1 | new_column |",
"+----+----+------------+",
"| a | a | true |",
"+----+----+------------+",
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn with_column_renamed() -> Result<()> {
let df = test_table()
.await?
.select_columns(&["c1", "c2", "c3"])?
.filter(col("c2").eq(lit(3)).and(col("c1").eq(lit("a"))))?
.sort(vec![
// make the test deterministic
col("c1").sort(true, true),
col("c2").sort(true, true),
col("c3").sort(true, true),
])?
.limit(0, Some(1))?
.with_column("sum", col("c2") + col("c3"))?;
let df_sum_renamed = df
.with_column_renamed("sum", "total")?
// table qualifier optional
.with_column_renamed("c1", "one")?
// accepts table qualifier
.with_column_renamed("aggregate_test_100.c2", "two")?
// no-op for missing column
.with_column_renamed("c4", "boom")?
.collect()
.await?;
assert_batches_sorted_eq!(
[
"+-----+-----+-----+-------+",
"| one | two | c3 | total |",
"+-----+-----+-----+-------+",
"| a | 3 | -72 | -69 |",
"+-----+-----+-----+-------+",
],
&df_sum_renamed
);
Ok(())
}
#[tokio::test]
async fn with_column_renamed_ambiguous() -> Result<()> {
let df = test_table().await?.select_columns(&["c1", "c2", "c3"])?;
let ctx = SessionContext::new();
let table = df.into_view();
ctx.register_table("t1", table.clone())?;
ctx.register_table("t2", table)?;
let actual_err = ctx
.table("t1")
.await?
.join(
ctx.table("t2").await?,
JoinType::Inner,
&["c1"],
&["c1"],
None,
)?
// can be t1.c2 or t2.c2
.with_column_renamed("c2", "AAA")
.unwrap_err();
let expected_err = "Schema error: Ambiguous reference to unqualified field c2";
assert_eq!(actual_err.strip_backtrace(), expected_err);
Ok(())
}
#[tokio::test]
async fn with_column_renamed_join() -> Result<()> {
let df = test_table().await?.select_columns(&["c1", "c2", "c3"])?;
let ctx = SessionContext::new();
let table = df.into_view();
ctx.register_table("t1", table.clone())?;
ctx.register_table("t2", table)?;
let df = ctx
.table("t1")
.await?
.join(
ctx.table("t2").await?,
JoinType::Inner,
&["c1"],
&["c1"],
None,
)?
.sort(vec![
// make the test deterministic
col("t1.c1").sort(true, true),
col("t1.c2").sort(true, true),
col("t1.c3").sort(true, true),
col("t2.c1").sort(true, true),
col("t2.c2").sort(true, true),
col("t2.c3").sort(true, true),
])?
.limit(0, Some(1))?;
let df_results = df.clone().collect().await?;
assert_batches_sorted_eq!(
[
"+----+----+-----+----+----+-----+",
"| c1 | c2 | c3 | c1 | c2 | c3 |",
"+----+----+-----+----+----+-----+",
"| a | 1 | -85 | a | 1 | -85 |",
"+----+----+-----+----+----+-----+"
],
&df_results
);
let df_renamed = df.clone().with_column_renamed("t1.c1", "AAA")?;
assert_eq!("\
Projection: t1.c1 AS AAA, t1.c2, t1.c3, t2.c1, t2.c2, t2.c3\
\n Limit: skip=0, fetch=1\
\n Sort: t1.c1 ASC NULLS FIRST, t1.c2 ASC NULLS FIRST, t1.c3 ASC NULLS FIRST, t2.c1 ASC NULLS FIRST, t2.c2 ASC NULLS FIRST, t2.c3 ASC NULLS FIRST\
\n Inner Join: t1.c1 = t2.c1\
\n TableScan: t1\
\n TableScan: t2",
format!("{}", df_renamed.logical_plan())
);
assert_eq!("\
Projection: t1.c1 AS AAA, t1.c2, t1.c3, t2.c1, t2.c2, t2.c3\
\n Sort: t1.c1 ASC NULLS FIRST, t1.c2 ASC NULLS FIRST, t1.c3 ASC NULLS FIRST, t2.c1 ASC NULLS FIRST, t2.c2 ASC NULLS FIRST, t2.c3 ASC NULLS FIRST, fetch=1\
\n Inner Join: t1.c1 = t2.c1\
\n SubqueryAlias: t1\
\n TableScan: aggregate_test_100 projection=[c1, c2, c3]\
\n SubqueryAlias: t2\
\n TableScan: aggregate_test_100 projection=[c1, c2, c3]",
format!("{}", df_renamed.clone().into_optimized_plan()?)
);
let df_results = df_renamed.collect().await?;
assert_batches_sorted_eq!(
[
"+-----+----+-----+----+----+-----+",
"| AAA | c2 | c3 | c1 | c2 | c3 |",
"+-----+----+-----+----+----+-----+",
"| a | 1 | -85 | a | 1 | -85 |",
"+-----+----+-----+----+----+-----+"
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn with_column_renamed_case_sensitive() -> Result<()> {
let config = SessionConfig::from_string_hash_map(&HashMap::from([(
"datafusion.sql_parser.enable_ident_normalization".to_owned(),
"false".to_owned(),
)]))?;
let ctx = SessionContext::new_with_config(config);
let name = "aggregate_test_100";
register_aggregate_csv(&ctx, name).await?;
let df = ctx.table(name);
let df = df
.await?
.filter(col("c2").eq(lit(3)).and(col("c1").eq(lit("a"))))?
.limit(0, Some(1))?
.sort(vec![
// make the test deterministic
col("c1").sort(true, true),
col("c2").sort(true, true),
col("c3").sort(true, true),
])?
.select_columns(&["c1"])?;
let df_renamed = df.clone().with_column_renamed("c1", "CoLuMn1")?;
let res = &df_renamed.clone().collect().await?;
assert_batches_sorted_eq!(
[
"+---------+",
"| CoLuMn1 |",
"+---------+",
"| a |",
"+---------+"
],
res
);
let df_renamed = df_renamed
.with_column_renamed("CoLuMn1", "c1")?
.collect()
.await?;
assert_batches_sorted_eq!(
["+----+", "| c1 |", "+----+", "| a |", "+----+"],
&df_renamed
);
Ok(())
}
#[tokio::test]
async fn cast_expr_test() -> Result<()> {
let df = test_table()
.await?
.select_columns(&["c2", "c3"])?
.limit(0, Some(1))?
.with_column("sum", cast(col("c2") + col("c3"), DataType::Int64))?;
let df_results = df.clone().collect().await?;
df.clone().show().await?;
assert_batches_sorted_eq!(
[
"+----+----+-----+",
"| c2 | c3 | sum |",
"+----+----+-----+",
"| 2 | 1 | 3 |",
"+----+----+-----+"
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn row_writer_resize_test() -> Result<()> {
let schema = Arc::new(Schema::new(vec![Field::new(
"column_1",
DataType::Utf8,
false,
)]));
let data = RecordBatch::try_new(
schema,
vec![
Arc::new(StringArray::from(vec![
Some("2a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"),
Some("3a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000800"),
]))
],
)?;
let ctx = SessionContext::new();
ctx.register_batch("test", data)?;
let sql = r#"
SELECT
count(1)
FROM
test
GROUP BY
column_1"#;
let df = ctx.sql(sql).await?;
df.show_limit(10).await?;
Ok(())
}
#[tokio::test]
async fn with_column_name() -> Result<()> {
// define data with a column name that has a "." in it:
let array: Int32Array = [1, 10].into_iter().collect();
let batch = RecordBatch::try_from_iter(vec![("f.c1", Arc::new(array) as _)])?;
let ctx = SessionContext::new();
ctx.register_batch("t", batch)?;
let df = ctx
.table("t")
.await?
// try and create a column with a '.' in it
.with_column("f.c2", lit("hello"))?;
let df_results = df.collect().await?;
assert_batches_sorted_eq!(
[
"+------+-------+",
"| f.c1 | f.c2 |",
"+------+-------+",
"| 1 | hello |",
"| 10 | hello |",
"+------+-------+"
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn test_cache_mismatch() -> Result<()> {
let ctx = SessionContext::new();
let df = ctx
.sql("SELECT CASE WHEN true THEN NULL ELSE 1 END")
.await?;
let cache_df = df.cache().await;
assert!(cache_df.is_ok());
Ok(())
}
#[tokio::test]
async fn cache_test() -> Result<()> {
let df = test_table()
.await?
.select_columns(&["c2", "c3"])?
.limit(0, Some(1))?
.with_column("sum", cast(col("c2") + col("c3"), DataType::Int64))?;
let cached_df = df.clone().cache().await?;
assert_eq!(
"TableScan: ?table? projection=[c2, c3, sum]",
format!("{}", cached_df.clone().into_optimized_plan()?)
);
let df_results = df.collect().await?;
let cached_df_results = cached_df.collect().await?;
assert_batches_sorted_eq!(
[
"+----+----+-----+",
"| c2 | c3 | sum |",
"+----+----+-----+",
"| 2 | 1 | 3 |",
"+----+----+-----+"
],
&cached_df_results
);
assert_eq!(&df_results, &cached_df_results);
Ok(())
}
#[tokio::test]
async fn partition_aware_union() -> Result<()> {
let left = test_table().await?.select_columns(&["c1", "c2"])?;
let right = test_table_with_name("c2")
.await?
.select_columns(&["c1", "c3"])?
.with_column_renamed("c2.c1", "c2_c1")?;
let left_rows = left.clone().collect().await?;
let right_rows = right.clone().collect().await?;
let join1 = left.clone().join(
right.clone(),
JoinType::Inner,
&["c1"],
&["c2_c1"],
None,
)?;
let join2 = left.join(right, JoinType::Inner, &["c1"], &["c2_c1"], None)?;
let union = join1.union(join2)?;
let union_rows = union.clone().collect().await?;
assert_eq!(100, left_rows.iter().map(|x| x.num_rows()).sum::<usize>());
assert_eq!(100, right_rows.iter().map(|x| x.num_rows()).sum::<usize>());
assert_eq!(4016, union_rows.iter().map(|x| x.num_rows()).sum::<usize>());
let physical_plan = union.create_physical_plan().await?;
let default_partition_count = SessionConfig::new().target_partitions();
// For partition aware union, the output partition count should not be changed.
assert_eq!(
physical_plan.output_partitioning().partition_count(),
default_partition_count
);
// For partition aware union, the output partition is the same with the union's inputs
for child in physical_plan.children() {
assert_eq!(
physical_plan.output_partitioning(),
child.output_partitioning()
);
}
Ok(())
}
#[tokio::test]
async fn non_partition_aware_union() -> Result<()> {
let left = test_table().await?.select_columns(&["c1", "c2"])?;
let right = test_table_with_name("c2")
.await?
.select_columns(&["c1", "c2"])?
.with_column_renamed("c2.c1", "c2_c1")?
.with_column_renamed("c2.c2", "c2_c2")?;
let left_rows = left.clone().collect().await?;
let right_rows = right.clone().collect().await?;
let join1 = left.clone().join(
right.clone(),
JoinType::Inner,
&["c1", "c2"],
&["c2_c1", "c2_c2"],
None,
)?;
// join key ordering is different
let join2 = left.join(
right,
JoinType::Inner,
&["c2", "c1"],
&["c2_c2", "c2_c1"],
None,
)?;
let union = join1.union(join2)?;
let union_rows = union.clone().collect().await?;
assert_eq!(100, left_rows.iter().map(|x| x.num_rows()).sum::<usize>());
assert_eq!(100, right_rows.iter().map(|x| x.num_rows()).sum::<usize>());
assert_eq!(916, union_rows.iter().map(|x| x.num_rows()).sum::<usize>());
let physical_plan = union.create_physical_plan().await?;
let default_partition_count = SessionConfig::new().target_partitions();
// For non-partition aware union, the output partitioning count should be the combination of all output partitions count
assert!(matches!(
physical_plan.output_partitioning(),
Partitioning::UnknownPartitioning(partition_count) if *partition_count == default_partition_count * 2));
Ok(())
}
#[tokio::test]
async fn verify_join_output_partitioning() -> Result<()> {
let left = test_table().await?.select_columns(&["c1", "c2"])?;
let right = test_table_with_name("c2")
.await?
.select_columns(&["c1", "c2"])?
.with_column_renamed("c2.c1", "c2_c1")?
.with_column_renamed("c2.c2", "c2_c2")?;
let all_join_types = vec![
JoinType::Inner,
JoinType::Left,
JoinType::Right,
JoinType::Full,
JoinType::LeftSemi,
JoinType::RightSemi,
JoinType::LeftAnti,
JoinType::RightAnti,
JoinType::LeftMark,
];
let default_partition_count = SessionConfig::new().target_partitions();
for join_type in all_join_types {
let join = left.clone().join(
right.clone(),
join_type,
&["c1", "c2"],
&["c2_c1", "c2_c2"],
None,
)?;
let physical_plan = join.create_physical_plan().await?;
let out_partitioning = physical_plan.output_partitioning();
let join_schema = physical_plan.schema();
match join_type {
JoinType::Left
| JoinType::LeftSemi
| JoinType::LeftAnti
| JoinType::LeftMark => {
let left_exprs: Vec<Arc<dyn PhysicalExpr>> = vec![
Arc::new(Column::new_with_schema("c1", &join_schema)?),
Arc::new(Column::new_with_schema("c2", &join_schema)?),
];
assert_eq!(
out_partitioning,
&Partitioning::Hash(left_exprs, default_partition_count)
);
}
JoinType::Inner
| JoinType::Right
| JoinType::RightSemi
| JoinType::RightAnti => {
let right_exprs: Vec<Arc<dyn PhysicalExpr>> = vec![
Arc::new(Column::new_with_schema("c2_c1", &join_schema)?),
Arc::new(Column::new_with_schema("c2_c2", &join_schema)?),
];
assert_eq!(
out_partitioning,
&Partitioning::Hash(right_exprs, default_partition_count)
);
}
JoinType::Full => {
assert!(matches!(
out_partitioning,
&Partitioning::UnknownPartitioning(partition_count) if partition_count == default_partition_count));
}
}
}
Ok(())
}
#[tokio::test]
async fn test_except_nested_struct() -> Result<()> {
use arrow::array::StructArray;
let nested_schema = Arc::new(Schema::new(vec![
Field::new("id", DataType::Int32, true),
Field::new("lat", DataType::Int32, true),
Field::new("long", DataType::Int32, true),
]));
let schema = Arc::new(Schema::new(vec![
Field::new("value", DataType::Int32, true),
Field::new(
"nested",
DataType::Struct(nested_schema.fields.clone()),
true,
),
]));
let batch = RecordBatch::try_new(
Arc::clone(&schema),
vec![
Arc::new(Int32Array::from(vec![Some(1), Some(2), Some(3)])),
Arc::new(StructArray::from(vec![
(
Arc::new(Field::new("id", DataType::Int32, true)),
Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef,
),
(
Arc::new(Field::new("lat", DataType::Int32, true)),
Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef,
),
(
Arc::new(Field::new("long", DataType::Int32, true)),
Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef,
),
])),
],
)
.unwrap();
let updated_batch = RecordBatch::try_new(
Arc::clone(&schema),
vec![
Arc::new(Int32Array::from(vec![Some(1), Some(12), Some(3)])),
Arc::new(StructArray::from(vec![
(
Arc::new(Field::new("id", DataType::Int32, true)),
Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef,
),
(
Arc::new(Field::new("lat", DataType::Int32, true)),
Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef,
),
(
Arc::new(Field::new("long", DataType::Int32, true)),
Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef,
),
])),
],
)
.unwrap();
let ctx = SessionContext::new();
let before = ctx.read_batch(batch).expect("Failed to make DataFrame");
let after = ctx
.read_batch(updated_batch)
.expect("Failed to make DataFrame");
let diff = before
.except(after)
.expect("Failed to except")
.collect()
.await?;
assert_eq!(diff.len(), 1);
Ok(())
}
#[tokio::test]
async fn nested_explain_should_fail() -> Result<()> {
let ctx = SessionContext::new();
// must be error
let mut result = ctx.sql("explain select 1").await?.explain(false, false);
assert!(result.is_err());
// must be error
result = ctx.sql("explain explain select 1").await;
assert!(result.is_err());
Ok(())
}
// Test issue: https://github.com/apache/datafusion/issues/12065
#[tokio::test]
async fn filtered_aggr_with_param_values() -> Result<()> {
let cfg = SessionConfig::new().set(
"datafusion.sql_parser.dialect",
&ScalarValue::from("PostgreSQL"),
);
let ctx = SessionContext::new_with_config(cfg);
register_aggregate_csv(&ctx, "table1").await?;
let df = ctx
.sql("select count (c2) filter (where c3 > $1) from table1")
.await?
.with_param_values(ParamValues::List(vec![ScalarValue::from(10u64)]));
let df_results = df?.collect().await?;
assert_batches_eq!(
&[
"+------------------------------------------------+",
"| count(table1.c2) FILTER (WHERE table1.c3 > $1) |",
"+------------------------------------------------+",
"| 54 |",
"+------------------------------------------------+",
],
&df_results
);
Ok(())
}
// Test issue: https://github.com/apache/datafusion/issues/13873
#[tokio::test]
async fn write_parquet_with_order() -> Result<()> {
let tmp_dir = TempDir::new()?;
let schema = Arc::new(Schema::new(vec![
Field::new("a", DataType::Int32, true),
Field::new("b", DataType::Int32, true),
]));
let ctx = SessionContext::new();
let write_df = ctx.read_batch(RecordBatch::try_new(
schema.clone(),
vec![
Arc::new(Int32Array::from(vec![1, 5, 7, 3, 2])),
Arc::new(Int32Array::from(vec![2, 3, 4, 5, 6])),
],
)?)?;
let test_path = tmp_dir.path().join("test.parquet");
write_df
.clone()
.write_parquet(
test_path.to_str().unwrap(),
DataFrameWriteOptions::new()
.with_sort_by(vec![col("a").sort(true, true)]),
None,
)
.await?;
let ctx = SessionContext::new();
ctx.register_parquet(
"data",
test_path.to_str().unwrap(),
ParquetReadOptions::default(),
)
.await?;
let df = ctx.sql("SELECT * FROM data").await?;
let results = df.collect().await?;
let df_explain = ctx.sql("explain SELECT a FROM data").await?;
let explain_result = df_explain.collect().await?;
println!("explain_result {:?}", explain_result);
assert_batches_eq!(
&[
"+---+---+",
"| a | b |",
"+---+---+",
"| 1 | 2 |",
"| 2 | 6 |",
"| 3 | 5 |",
"| 5 | 3 |",
"| 7 | 4 |",
"+---+---+",
],
&results
);
Ok(())
}
// Test issue: https://github.com/apache/datafusion/issues/13873
#[tokio::test]
async fn write_csv_with_order() -> Result<()> {
let tmp_dir = TempDir::new()?;
let schema = Arc::new(Schema::new(vec![
Field::new("a", DataType::Int32, true),
Field::new("b", DataType::Int32, true),
]));
let ctx = SessionContext::new();
let write_df = ctx.read_batch(RecordBatch::try_new(
schema.clone(),
vec![
Arc::new(Int32Array::from(vec![1, 5, 7, 3, 2])),
Arc::new(Int32Array::from(vec![2, 3, 4, 5, 6])),
],
)?)?;
let test_path = tmp_dir.path().join("test.csv");
write_df
.clone()
.write_csv(
test_path.to_str().unwrap(),
DataFrameWriteOptions::new()
.with_sort_by(vec![col("a").sort(true, true)]),
None,
)
.await?;
let ctx = SessionContext::new();
ctx.register_csv(
"data",
test_path.to_str().unwrap(),
CsvReadOptions::new().schema(&schema),
)
.await?;
let df = ctx.sql("SELECT * FROM data").await?;
let results = df.collect().await?;
assert_batches_eq!(
&[
"+---+---+",
"| a | b |",
"+---+---+",
"| 1 | 2 |",
"| 2 | 6 |",
"| 3 | 5 |",
"| 5 | 3 |",
"| 7 | 4 |",
"+---+---+",
],
&results
);
Ok(())
}
// Test issue: https://github.com/apache/datafusion/issues/13873
#[tokio::test]
async fn write_json_with_order() -> Result<()> {
let tmp_dir = TempDir::new()?;
let schema = Arc::new(Schema::new(vec![
Field::new("a", DataType::Int32, true),
Field::new("b", DataType::Int32, true),
]));
let ctx = SessionContext::new();
let write_df = ctx.read_batch(RecordBatch::try_new(
schema.clone(),
vec![
Arc::new(Int32Array::from(vec![1, 5, 7, 3, 2])),
Arc::new(Int32Array::from(vec![2, 3, 4, 5, 6])),
],
)?)?;
let test_path = tmp_dir.path().join("test.json");
write_df
.clone()
.write_json(
test_path.to_str().unwrap(),
DataFrameWriteOptions::new()
.with_sort_by(vec![col("a").sort(true, true)]),
None,
)
.await?;
let ctx = SessionContext::new();
ctx.register_json(
"data",
test_path.to_str().unwrap(),
NdJsonReadOptions::default().schema(&schema),
)
.await?;
let df = ctx.sql("SELECT * FROM data").await?;
let results = df.collect().await?;
assert_batches_eq!(
&[
"+---+---+",
"| a | b |",
"+---+---+",
"| 1 | 2 |",
"| 2 | 6 |",
"| 3 | 5 |",
"| 5 | 3 |",
"| 7 | 4 |",
"+---+---+",
],
&results
);
Ok(())
}
// Test issue: https://github.com/apache/datafusion/issues/13873
#[tokio::test]
async fn write_table_with_order() -> Result<()> {
let tmp_dir = TempDir::new()?;
let ctx = SessionContext::new();
let location = tmp_dir.path().join("test_table/");
let mut write_df = ctx
.sql("values ('z'), ('x'), ('a'), ('b'), ('c')")
.await
.unwrap();
// Ensure the column names and types match the target table
write_df = write_df
.with_column_renamed("column1", "tablecol1")
.unwrap();
let sql_str =
"create external table data(tablecol1 varchar) stored as parquet location '"
.to_owned()
+ location.to_str().unwrap()
+ "'";
ctx.sql(sql_str.as_str()).await?.collect().await?;
// This is equivalent to INSERT INTO test.
write_df
.clone()
.write_table(
"data",
DataFrameWriteOptions::new()
.with_sort_by(vec![col("tablecol1").sort(true, true)]),
)
.await?;
let df = ctx.sql("SELECT * FROM data").await?;
let results = df.collect().await?;
assert_batches_eq!(
&[
"+-----------+",
"| tablecol1 |",
"+-----------+",
"| a |",
"| b |",
"| c |",
"| x |",
"| z |",
"+-----------+",
],
&results
);
Ok(())
}
}