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
// 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.
use std::any::Any;
use std::sync::Arc;
use arrow::array::{Array, ArrayRef, Int64Array, StringArray};
use arrow::compute::{cast, concat};
use arrow::datatypes::{DataType, Field};
use async_trait::async_trait;
use datafusion_common::{DataFusionError, SchemaError};
use parquet::file::properties::WriterProperties;
use datafusion_common::{Column, DFSchema, ScalarValue};
use datafusion_expr::{
avg, count, is_null, max, median, min, stddev, utils::COUNT_STAR_EXPANSION,
TableProviderFilterPushDown, UNNAMED_TABLE,
};
use crate::arrow::datatypes::Schema;
use crate::arrow::datatypes::SchemaRef;
use crate::arrow::record_batch::RecordBatch;
use crate::arrow::util::pretty;
use crate::datasource::physical_plan::{plan_to_csv, plan_to_json, plan_to_parquet};
use crate::datasource::{provider_as_source, MemTable, TableProvider};
use crate::error::Result;
use crate::execution::{
context::{SessionState, TaskContext},
FunctionRegistry,
};
use crate::logical_expr::{
col, utils::find_window_exprs, Expr, JoinType, LogicalPlan, LogicalPlanBuilder,
Partitioning, TableType,
};
use crate::physical_plan::SendableRecordBatchStream;
use crate::physical_plan::{collect, collect_partitioned};
use crate::physical_plan::{execute_stream, execute_stream_partitioned, ExecutionPlan};
use crate::prelude::SessionContext;
/// DataFrame represents a logical set of rows with the same named columns.
/// Similar to a [Pandas DataFrame](https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html) or
/// [Spark DataFrame](https://spark.apache.org/docs/latest/sql-programming-guide.html)
///
/// DataFrames are typically created by the `read_csv` and `read_parquet` methods on the
/// [SessionContext](../execution/context/struct.SessionContext.html) and can then be modified
/// by calling the transformation methods, such as `filter`, `select`, `aggregate`, and `limit`
/// to build up a query definition.
///
/// The query can be executed by calling the `collect` method.
///
/// ```
/// # 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.filter(col("a").lt_eq(col("b")))?
/// .aggregate(vec![col("a")], vec![min(col("b"))])?
/// .limit(0, Some(100))?;
/// let results = df.collect();
/// # Ok(())
/// # }
/// ```
#[derive(Debug, Clone)]
pub struct DataFrame {
session_state: SessionState,
plan: LogicalPlan,
}
impl DataFrame {
/// Create a new Table based on an existing logical plan
pub fn new(session_state: SessionState, plan: LogicalPlan) -> Self {
Self {
session_state,
plan,
}
}
/// Create 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;
/// # #[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"])?;
/// # Ok(())
/// # }
/// ```
pub fn select_columns(self, columns: &[&str]) -> Result<DataFrame> {
let fields = columns
.iter()
.map(|name| self.plan.schema().field_with_unqualified_name(name))
.collect::<Result<Vec<_>>>()?;
let expr: Vec<Expr> = fields
.iter()
.map(|f| Expr::Column(f.qualified_column()))
.collect();
self.select(expr)
}
/// Create a projection based on arbitrary expressions.
///
/// ```
/// # 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.select(vec![col("a") * col("b"), col("c")])?;
/// # 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::new(self.session_state, project_plan))
}
/// Expand each list element of a column to multiple rows.
///
/// ```
/// # 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.unnest_column("a")?;
/// # Ok(())
/// # }
/// ```
pub fn unnest_column(self, column: &str) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.unnest_column(column)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Filter a DataFrame to only include rows that match the specified filter expression.
///
/// ```
/// # 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.filter(col("a").lt_eq(col("b")))?;
/// # Ok(())
/// # }
/// ```
pub fn filter(self, predicate: Expr) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.filter(predicate)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Perform an aggregate query with optional grouping expressions.
///
/// ```
/// # 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?;
///
/// // The following use is the equivalent of "SELECT MIN(b) GROUP BY a"
/// let _ = df.clone().aggregate(vec![col("a")], vec![min(col("b"))])?;
///
/// // The following use is the equivalent of "SELECT MIN(b)"
/// let _ = df.aggregate(vec![], vec![min(col("b"))])?;
/// # Ok(())
/// # }
/// ```
pub fn aggregate(
self,
group_expr: Vec<Expr>,
aggr_expr: Vec<Expr>,
) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.aggregate(group_expr, aggr_expr)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Apply one or more window functions ([`Expr::WindowFunction`]) to extend the schema
pub fn window(self, window_exprs: Vec<Expr>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.window(window_exprs)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Limit the number of rows returned from this DataFrame.
///
/// `skip` - Number of rows to skip before fetch any row
///
/// `fetch` - Maximum number of rows to fetch, after skipping `skip` rows.
/// ```
/// # 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.limit(0, Some(100))?;
/// # Ok(())
/// # }
/// ```
pub fn limit(self, skip: usize, fetch: Option<usize>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.limit(skip, fetch)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Calculate the union of two [`DataFrame`]s, preserving duplicate rows.The
/// two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # 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 d2 = df.clone();
/// let df = df.union(d2)?;
/// # Ok(())
/// # }
/// ```
pub fn union(self, dataframe: DataFrame) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.union(dataframe.plan)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Calculate the distinct union of two [`DataFrame`]s. The
/// two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # 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 d2 = df.clone();
/// let df = df.union_distinct(d2)?;
/// # Ok(())
/// # }
/// ```
pub fn union_distinct(self, dataframe: DataFrame) -> Result<DataFrame> {
Ok(DataFrame::new(
self.session_state,
LogicalPlanBuilder::from(self.plan)
.union_distinct(dataframe.plan)?
.build()?,
))
}
/// Filter out duplicate rows
///
/// ```
/// # 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.distinct()?;
/// # Ok(())
/// # }
/// ```
pub fn distinct(self) -> Result<DataFrame> {
Ok(DataFrame::new(
self.session_state,
LogicalPlanBuilder::from(self.plan).distinct()?.build()?,
))
}
/// Summary statistics for a DataFrame. Only summarizes numeric datatypes at the moment and
/// returns nulls for non numeric datatypes. Try in keep output similar to pandas
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use arrow::util::pretty;
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let df = ctx.read_csv("tests/tpch-csv/customer.csv", CsvReadOptions::new()).await?;
/// df.describe().await.unwrap();
///
/// # 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| count(is_null(col(f.name()))).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 batchs = df.clone().collect().await;
match batchs {
Ok(batchs)
if batchs.len() == 1
&& batchs[0]
.column_by_name(field.name())
.is_some() =>
{
let column =
batchs[0].column_by_name(field.name()).unwrap();
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(other_err) => {
panic!("{other_err}")
}
};
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]],
)?;
Ok(DataFrame::new(
self.session_state,
LogicalPlanBuilder::scan(
UNNAMED_TABLE,
provider_as_source(Arc::new(provider)),
None,
)?
.build()?,
))
}
/// Sort the DataFrame by the specified sorting expressions. Any expression can be turned into
/// a sort expression by calling its [sort](../logical_plan/enum.Expr.html#method.sort) method.
///
/// ```
/// # 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.sort(vec![col("a").sort(true, true), col("b").sort(false, false)])?;
/// # Ok(())
/// # }
/// ```
pub fn sort(self, expr: Vec<Expr>) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan).sort(expr)?.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Join this DataFrame with another DataFrame using the specified columns as join keys.
///
/// Filter expression expected to contain non-equality predicates that can not be pushed
/// down to any of join inputs.
/// In case of outer join, filter applied to only matched rows.
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[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")])?;
/// let join = left.join(right, JoinType::Inner, &["a", "b"], &["a2", "b2"], None)?;
/// let batches = 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::new(self.session_state, plan))
}
/// Join this DataFrame with another DataFrame using the specified expressions.
///
/// Simply a thin wrapper over [`join`](Self::join) where the join keys are not provided,
/// and the provided expressions are AND'ed together to form the filter expression.
///
/// ```
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # #[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"),
/// ])?;
/// let join_on = left.join_on(
/// right,
/// JoinType::Inner,
/// [col("a").not_eq(col("a2")), col("b").not_eq(col("b2"))],
/// )?;
/// let batches = join_on.collect().await?;
/// # Ok(())
/// # }
/// ```
pub fn join_on(
self,
right: DataFrame,
join_type: JoinType,
on_exprs: impl IntoIterator<Item = Expr>,
) -> Result<DataFrame> {
let expr = on_exprs.into_iter().reduce(Expr::and);
let plan = LogicalPlanBuilder::from(self.plan)
.join(
right.plan,
join_type,
(Vec::<Column>::new(), Vec::<Column>::new()),
expr,
)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Repartition a DataFrame based on a logical partitioning scheme.
///
/// ```
/// # 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 df1 = df.repartition(Partitioning::RoundRobinBatch(4))?;
/// # Ok(())
/// # }
/// ```
pub fn repartition(self, partitioning_scheme: Partitioning) -> Result<DataFrame> {
let plan = LogicalPlanBuilder::from(self.plan)
.repartition(partitioning_scheme)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Run a count aggregate on the DataFrame and execute the DataFrame to collect this
/// count and return it as a usize, to find the total number of rows after executing
/// the DataFrame.
/// ```
/// # 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?;
/// # Ok(())
/// # }
/// ```
pub async fn count(self) -> Result<usize> {
let rows = self
.aggregate(
vec![],
vec![datafusion_expr::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)
}
/// Convert the logical plan represented by this DataFrame into a physical plan and
/// execute it, collecting all resulting batches into memory
/// Executes this DataFrame and collects all results into a vector of RecordBatch.
/// ```
/// # 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
}
/// Print results.
///
/// ```
/// # 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)?)
}
/// Print results and limit rows.
///
/// ```
/// # 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)?)
}
fn task_ctx(&self) -> TaskContext {
TaskContext::from(&self.session_state)
}
/// Executes this DataFrame and returns a stream over a single partition
///
/// ```
/// # 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(())
/// # }
/// ```
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.
///
/// ```
/// # 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.
///
/// ```
/// # 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(())
/// # }
/// ```
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 schema describing the output of this DataFrame in terms of columns returned,
/// where each column has a name, data type, and nullability attribute.
/// ```
/// # 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 the unoptimized logical plan
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 logical plan represented by this DataFrame without running the 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
pub fn into_unoptimized_plan(self) -> LogicalPlan {
self.plan
}
/// Return the optimized logical plan represented by this DataFrame.
///
/// 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
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 the optimized logical plan represented by this DataFrame.
///
/// 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
#[deprecated(since = "23.0.0", note = "Use DataFrame::into_optimized_plan")]
pub fn to_logical_plan(self) -> Result<LogicalPlan> {
self.into_optimized_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> {
let plan = LogicalPlanBuilder::from(self.plan)
.explain(verbose, analyze)?
.build()?;
Ok(DataFrame::new(self.session_state, plan))
}
/// Return a `FunctionRegistry` used to plan udf's calls
///
/// ```
/// # 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
}
/// Calculate the intersection of two [`DataFrame`]s. The two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # 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 d2 = df.clone();
/// let df = df.intersect(d2)?;
/// # Ok(())
/// # }
/// ```
pub fn intersect(self, dataframe: DataFrame) -> Result<DataFrame> {
let left_plan = self.plan;
let right_plan = dataframe.plan;
Ok(DataFrame::new(
self.session_state,
LogicalPlanBuilder::intersect(left_plan, right_plan, true)?,
))
}
/// Calculate the exception of two [`DataFrame`]s. The two [`DataFrame`]s must have exactly the same schema
///
/// ```
/// # 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 d2 = df.clone();
/// let df = df.except(d2)?;
/// # Ok(())
/// # }
/// ```
pub fn except(self, dataframe: DataFrame) -> Result<DataFrame> {
let left_plan = self.plan;
let right_plan = dataframe.plan;
Ok(DataFrame::new(
self.session_state,
LogicalPlanBuilder::except(left_plan, right_plan, true)?,
))
}
/// Write a `DataFrame` to a CSV file.
pub async fn write_csv(self, path: &str) -> Result<()> {
let plan = self.session_state.create_physical_plan(&self.plan).await?;
let task_ctx = Arc::new(self.task_ctx());
plan_to_csv(task_ctx, plan, path).await
}
/// Write a `DataFrame` to a Parquet file.
pub async fn write_parquet(
self,
path: &str,
writer_properties: Option<WriterProperties>,
) -> Result<()> {
let plan = self.session_state.create_physical_plan(&self.plan).await?;
let task_ctx = Arc::new(self.task_ctx());
plan_to_parquet(task_ctx, plan, path, writer_properties).await
}
/// Executes a query and writes the results to a partitioned JSON file.
pub async fn write_json(self, path: impl AsRef<str>) -> Result<()> {
let plan = self.session_state.create_physical_plan(&self.plan).await?;
let task_ctx = Arc::new(self.task_ctx());
plan_to_json(task_ctx, plan, path).await
}
/// Add an additional column to the DataFrame.
///
/// ```
/// # 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(&[expr.clone()]);
let plan = if window_func_exprs.is_empty() {
self.plan
} else {
LogicalPlanBuilder::window_plan(self.plan, window_func_exprs)?
};
let new_column = expr.alias(name);
let mut col_exists = false;
let mut fields: Vec<Expr> = plan
.schema()
.fields()
.iter()
.map(|f| {
if f.name() == name {
col_exists = true;
new_column.clone()
} else {
Expr::Column(Column {
relation: None,
name: f.name().into(),
})
}
})
.collect();
if !col_exists {
fields.push(new_column);
}
let project_plan = LogicalPlanBuilder::from(plan).project(fields)?.build()?;
Ok(DataFrame::new(self.session_state, project_plan))
}
/// Rename one column by applying a new projection. This is a no-op if the column to be
/// renamed does not exist.
///
/// ```
/// # 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<Column>,
new_name: &str,
) -> Result<DataFrame> {
let old_name: Column = old_name.into();
let field_to_rename = match self.plan.schema().field_from_column(&old_name) {
Ok(field) => 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()
.fields()
.iter()
.map(|f| {
if f == field_to_rename {
col(f.qualified_column()).alias(new_name)
} else {
col(f.qualified_column())
}
})
.collect::<Vec<_>>();
let project_plan = LogicalPlanBuilder::from(self.plan)
.project(projection)?
.build()?;
Ok(DataFrame::new(self.session_state, project_plan))
}
/// Convert a prepare logical plan into its inner logical plan with all params replaced with their corresponding values
pub fn with_param_values(self, param_values: Vec<ScalarValue>) -> Result<Self> {
let plan = self.plan.with_param_values(param_values)?;
Ok(Self::new(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::with_state(self.session_state.clone());
let mem_table = MemTable::try_new(
SchemaRef::from(self.schema().clone()),
self.collect_partitioned().await?,
)?;
context.read_table(Arc::new(mem_table))
}
}
struct DataFrameTableProvider {
plan: LogicalPlan,
}
#[async_trait]
impl TableProvider for DataFrameTableProvider {
fn as_any(&self) -> &dyn Any {
self
}
fn get_logical_plan(&self) -> Option<&LogicalPlan> {
Some(&self.plan)
}
fn supports_filter_pushdown(
&self,
_filter: &Expr,
) -> Result<TableProviderFilterPushDown> {
// A filter is added on the DataFrame when given
Ok(TableProviderFilterPushDown::Exact)
}
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: &SessionState,
projection: Option<&Vec<usize>>,
filters: &[Expr],
limit: Option<usize>,
) -> Result<Arc<dyn ExecutionPlan>> {
let mut expr = LogicalPlanBuilder::from(self.plan.clone());
if let Some(p) = projection {
expr = expr.select(p.iter().copied())?
}
// Add filter when given
let filter = filters.iter().cloned().reduce(|acc, new| acc.and(new));
if let Some(filter) = filter {
expr = expr.filter(filter)?
}
// 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 arrow::array::Int32Array;
use arrow::datatypes::DataType;
use datafusion_expr::{
avg, cast, count, count_distinct, create_udf, expr, lit, max, min, sum,
BuiltInWindowFunction, ScalarFunctionImplementation, Volatility, WindowFrame,
WindowFunction,
};
use datafusion_physical_expr::expressions::Column;
use crate::execution::context::SessionConfig;
use crate::execution::options::{CsvReadOptions, ParquetReadOptions};
use crate::physical_plan::ColumnarValue;
use crate::physical_plan::Partitioning;
use crate::physical_plan::PhysicalExpr;
use crate::test_util;
use crate::test_util::parquet_test_data;
use crate::{assert_batches_sorted_eq, execution::context::SessionContext};
use super::*;
#[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_with_window_exprs() -> Result<()> {
// build plan using Table API
let t = test_table().await?;
let first_row = Expr::WindowFunction(expr::WindowFunction::new(
WindowFunction::BuiltInWindowFunction(BuiltInWindowFunction::FirstValue),
vec![col("aggregate_test_100.c1")],
vec![col("aggregate_test_100.c2")],
vec![],
WindowFrame::new(false),
));
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!(
vec!["+------+", "| f.c1 |", "+------+", "| 1 |", "| 10 |", "+------+",],
&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!(
vec![
"+----+-----------------------------+-----------------------------+-----------------------------+-----------------------------+-------------------------------+----------------------------------------+",
"| 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 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!(
vec![
"+----+",
"| 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.to_string(), "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_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.to_string(), 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 mut ctx = SessionContext::new();
register_aggregate_csv(&mut 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],
Arc::new(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 = tokio::task::spawn(async move {
df.select_columns(&["c1"])
.expect("should be usable in a task")
});
task.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!(
vec![
"+----+-----------------------------+",
"| 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!(
vec![
"+----+---------------------+",
"| 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 mut ctx = SessionContext::new();
register_aggregate_csv(&mut ctx, "aggregate_test_100").await?;
Ok(ctx.sql(sql).await?.into_unoptimized_plan())
}
async fn test_table_with_name(name: &str) -> Result<DataFrame> {
let mut ctx = SessionContext::new();
register_aggregate_csv(&mut ctx, name).await?;
ctx.table(name).await
}
async fn test_table() -> Result<DataFrame> {
test_table_with_name("aggregate_test_100").await
}
async fn register_aggregate_csv(
ctx: &mut SessionContext,
table_name: &str,
) -> Result<()> {
let schema = test_util::aggr_test_schema();
let testdata = test_util::arrow_test_data();
ctx.register_csv(
table_name,
&format!("{testdata}/csv/aggregate_test_100.csv"),
CsvReadOptions::new().schema(schema.as_ref()),
)
.await?;
Ok(())
}
#[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!(
vec![
"+----+----+-----+-----+",
"| 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 ovwewritten
let df_results_overwrite = df
.clone()
.with_column("c1", col("c2") + col("c3"))?
.collect()
.await?;
assert_batches_sorted_eq!(
vec![
"+-----+----+-----+-----+",
"| 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 ovwewritten 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!(
vec![
"+----+----+-----+-----+",
"| 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(())
}
#[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"))))?
.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),
])?
.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!(
vec![
"+-----+-----+----+-------+",
"| one | two | c3 | total |",
"+-----+-----+----+-------+",
"| a | 3 | 13 | 16 |",
"+-----+-----+----+-------+",
],
&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.to_string(), 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!(
vec![
"+----+----+-----+----+----+-----+",
"| 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 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, 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!(
vec![
"+-----+----+-----+----+----+-----+",
"| AAA | c2 | c3 | c1 | c2 | c3 |",
"+-----+----+-----+----+----+-----+",
"| a | 1 | -85 | a | 1 | -85 |",
"+-----+----+-----+----+----+-----+",
],
&df_results
);
Ok(())
}
#[tokio::test]
async fn filter_pushdown_dataframe() -> Result<()> {
let ctx = SessionContext::new();
ctx.register_parquet(
"test",
&format!("{}/alltypes_plain.snappy.parquet", parquet_test_data()),
ParquetReadOptions::default(),
)
.await?;
ctx.register_table("t1", ctx.table("test").await?.into_view())?;
let df = ctx
.table("t1")
.await?
.filter(col("id").eq(lit(1)))?
.select_columns(&["bool_col", "int_col"])?;
let plan = df.explain(false, false)?.collect().await?;
// Filters all the way to Parquet
let formatted = pretty::pretty_format_batches(&plan)?.to_string();
assert!(formatted.contains("FilterExec: id@0 = 1"));
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!(
vec![
"+----+----+-----+",
"| 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![arrow::datatypes::Field::new(
"column_1",
DataType::Utf8,
false,
)]));
let data = RecordBatch::try_new(
schema,
vec![
Arc::new(arrow::array::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!(
vec![
"+------+-------+",
"| f.c1 | f.c2 |",
"+------+-------+",
"| 1 | hello |",
"| 10 | hello |",
"+------+-------+",
],
&df_results
);
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!(
vec![
"+----+----+-----+",
"| 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,
];
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::Inner
| JoinType::Left
| JoinType::LeftSemi
| JoinType::LeftAnti => {
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::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(())
}
}