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
// 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.
//! The table implementation.
use std::str::FromStr;
use std::{any::Any, sync::Arc};
use arrow::compute::SortOptions;
use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef};
use arrow_schema::Schema;
use async_trait::async_trait;
use dashmap::DashMap;
use datafusion_common::ToDFSchema;
use datafusion_expr::expr::Sort;
use datafusion_optimizer::utils::conjunction;
use datafusion_physical_expr::{create_physical_expr, LexOrdering, PhysicalSortExpr};
use futures::{future, stream, StreamExt, TryStreamExt};
use object_store::path::Path;
use object_store::ObjectMeta;
use crate::datasource::file_format::file_type::{FileCompressionType, FileType};
use crate::datasource::physical_plan::{FileScanConfig, FileSinkConfig};
use crate::datasource::{
file_format::{
arrow::ArrowFormat, avro::AvroFormat, csv::CsvFormat, json::JsonFormat,
parquet::ParquetFormat, FileFormat,
},
get_statistics_with_limit,
listing::ListingTableUrl,
TableProvider, TableType,
};
use crate::logical_expr::TableProviderFilterPushDown;
use crate::physical_plan;
use crate::{
error::{DataFusionError, Result},
execution::context::SessionState,
logical_expr::Expr,
physical_plan::{empty::EmptyExec, project_schema, ExecutionPlan, Statistics},
};
use super::PartitionedFile;
use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files};
/// Configuration for creating a [`ListingTable`]
#[derive(Debug, Clone)]
pub struct ListingTableConfig {
/// Paths on the `ObjectStore` for creating `ListingTable`.
/// They should share the same schema and object store.
pub table_paths: Vec<ListingTableUrl>,
/// Optional `SchemaRef` for the to be created `ListingTable`.
pub file_schema: Option<SchemaRef>,
/// Optional `ListingOptions` for the to be created `ListingTable`.
pub options: Option<ListingOptions>,
}
impl ListingTableConfig {
/// Creates new [`ListingTableConfig`].
///
/// The [`SchemaRef`] and [`ListingOptions`] are inferred based on
/// the suffix of the provided `table_paths` first element.
pub fn new(table_path: ListingTableUrl) -> Self {
let table_paths = vec![table_path];
Self {
table_paths,
file_schema: None,
options: None,
}
}
/// Creates new [`ListingTableConfig`] with multiple table paths.
///
/// The [`SchemaRef`] and [`ListingOptions`] are inferred based on
/// the suffix of the provided `table_paths` first element.
pub fn new_with_multi_paths(table_paths: Vec<ListingTableUrl>) -> Self {
Self {
table_paths,
file_schema: None,
options: None,
}
}
/// Add `schema` to [`ListingTableConfig`]
pub fn with_schema(self, schema: SchemaRef) -> Self {
Self {
table_paths: self.table_paths,
file_schema: Some(schema),
options: self.options,
}
}
/// Add `listing_options` to [`ListingTableConfig`]
pub fn with_listing_options(self, listing_options: ListingOptions) -> Self {
Self {
table_paths: self.table_paths,
file_schema: self.file_schema,
options: Some(listing_options),
}
}
fn infer_format(path: &str) -> Result<(Arc<dyn FileFormat>, String)> {
let err_msg = format!("Unable to infer file type from path: {path}");
let mut exts = path.rsplit('.');
let mut splitted = exts.next().unwrap_or("");
let file_compression_type = FileCompressionType::from_str(splitted)
.unwrap_or(FileCompressionType::UNCOMPRESSED);
if file_compression_type.is_compressed() {
splitted = exts.next().unwrap_or("");
}
let file_type = FileType::from_str(splitted)
.map_err(|_| DataFusionError::Internal(err_msg.to_owned()))?;
let ext = file_type
.get_ext_with_compression(file_compression_type.to_owned())
.map_err(|_| DataFusionError::Internal(err_msg))?;
let file_format: Arc<dyn FileFormat> = match file_type {
FileType::ARROW => Arc::new(ArrowFormat),
FileType::AVRO => Arc::new(AvroFormat),
FileType::CSV => Arc::new(
CsvFormat::default().with_file_compression_type(file_compression_type),
),
FileType::JSON => Arc::new(
JsonFormat::default().with_file_compression_type(file_compression_type),
),
FileType::PARQUET => Arc::new(ParquetFormat::default()),
};
Ok((file_format, ext))
}
/// Infer `ListingOptions` based on `table_path` suffix.
pub async fn infer_options(self, state: &SessionState) -> Result<Self> {
let store = if let Some(url) = self.table_paths.get(0) {
state.runtime_env().object_store(url)?
} else {
return Ok(self);
};
let file = self
.table_paths
.get(0)
.unwrap()
.list_all_files(store.as_ref(), "")
.next()
.await
.ok_or_else(|| DataFusionError::Internal("No files for table".into()))??;
let (format, file_extension) =
ListingTableConfig::infer_format(file.location.as_ref())?;
let listing_options = ListingOptions::new(format)
.with_file_extension(file_extension)
.with_target_partitions(state.config().target_partitions());
Ok(Self {
table_paths: self.table_paths,
file_schema: self.file_schema,
options: Some(listing_options),
})
}
/// Infer the [`SchemaRef`] based on `table_path` suffix. Requires `self.options` to be set prior to using.
pub async fn infer_schema(self, state: &SessionState) -> Result<Self> {
match self.options {
Some(options) => {
let schema = if let Some(url) = self.table_paths.get(0) {
options.infer_schema(state, url).await?
} else {
Arc::new(Schema::empty())
};
Ok(Self {
table_paths: self.table_paths,
file_schema: Some(schema),
options: Some(options),
})
}
None => Err(DataFusionError::Internal(
"No `ListingOptions` set for inferring schema".into(),
)),
}
}
/// Convenience wrapper for calling `infer_options` and `infer_schema`
pub async fn infer(self, state: &SessionState) -> Result<Self> {
self.infer_options(state).await?.infer_schema(state).await
}
}
/// Options for creating a [`ListingTable`]
#[derive(Clone, Debug)]
pub struct ListingOptions {
/// A suffix on which files should be filtered (leave empty to
/// keep all files on the path)
pub file_extension: String,
/// The file format
pub format: Arc<dyn FileFormat>,
/// The expected partition column names in the folder structure.
/// See [Self::with_table_partition_cols] for details
pub table_partition_cols: Vec<(String, DataType)>,
/// Set true to try to guess statistics from the files.
/// This can add a lot of overhead as it will usually require files
/// to be opened and at least partially parsed.
pub collect_stat: bool,
/// Group files to avoid that the number of partitions exceeds
/// this limit
pub target_partitions: usize,
/// Optional pre-known sort order(s). Must be `SortExpr`s.
///
/// DataFusion may take advantage of this ordering to omit sorts
/// or use more efficient algorithms. Currently sortedness must be
/// provided if it is known by some external mechanism, but may in
/// the future be automatically determined, for example using
/// parquet metadata.
///
/// See <https://github.com/apache/arrow-datafusion/issues/4177>
/// NOTE: This attribute stores all equivalent orderings (the outer `Vec`)
/// where each ordering consists of an individual lexicographic
/// ordering (encapsulated by a `Vec<Expr>`). If there aren't
/// multiple equivalent orderings, the outer `Vec` will have a
/// single element.
pub file_sort_order: Vec<Vec<Expr>>,
/// Infinite source means that the input is not guaranteed to end.
/// Currently, CSV, JSON, and AVRO formats are supported.
/// In order to support infinite inputs, DataFusion may adjust query
/// plans (e.g. joins) to run the given query in full pipelining mode.
pub infinite_source: bool,
}
impl ListingOptions {
/// Creates an options instance with the given format
/// Default values:
/// - no file extension filter
/// - no input partition to discover
/// - one target partition
/// - stat collection
pub fn new(format: Arc<dyn FileFormat>) -> Self {
Self {
file_extension: String::new(),
format,
table_partition_cols: vec![],
collect_stat: true,
target_partitions: 1,
file_sort_order: vec![],
infinite_source: false,
}
}
/// Set unbounded assumption on [`ListingOptions`] and returns self.
///
/// ```
/// use std::sync::Arc;
/// use datafusion::datasource::{listing::ListingOptions, file_format::csv::CsvFormat};
/// use datafusion::prelude::SessionContext;
/// let ctx = SessionContext::new();
/// let listing_options = ListingOptions::new(Arc::new(
/// CsvFormat::default()
/// )).with_infinite_source(true);
///
/// assert_eq!(listing_options.infinite_source, true);
/// ```
pub fn with_infinite_source(mut self, infinite_source: bool) -> Self {
self.infinite_source = infinite_source;
self
}
/// Set file extension on [`ListingOptions`] and returns self.
///
/// ```
/// # use std::sync::Arc;
/// # use datafusion::prelude::SessionContext;
/// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat};
///
/// let listing_options = ListingOptions::new(Arc::new(
/// ParquetFormat::default()
/// ))
/// .with_file_extension(".parquet");
///
/// assert_eq!(listing_options.file_extension, ".parquet");
/// ```
pub fn with_file_extension(mut self, file_extension: impl Into<String>) -> Self {
self.file_extension = file_extension.into();
self
}
/// Set `table partition columns` on [`ListingOptions`] and returns self.
///
/// "partition columns," used to support [Hive Partitioning], are
/// columns added to the data that is read, based on the folder
/// structure where the data resides.
///
/// For example, give the following files in your filesystem:
///
/// ```text
/// /mnt/nyctaxi/year=2022/month=01/tripdata.parquet
/// /mnt/nyctaxi/year=2021/month=12/tripdata.parquet
/// /mnt/nyctaxi/year=2021/month=11/tripdata.parquet
/// ```
///
/// A [`ListingTable`] created at `/mnt/nyctaxi/` with partition
/// columns "year" and "month" will include new `year` and `month`
/// columns while reading the files. The `year` column would have
/// value `2022` and the `month` column would have value `01` for
/// the rows read from
/// `/mnt/nyctaxi/year=2022/month=01/tripdata.parquet`
///
///# Notes
///
/// - If only one level (e.g. `year` in the example above) is
/// specified, the other levels are ignored but the files are
/// still read.
///
/// - Files that don't follow this partitioning scheme will be
/// ignored.
///
/// - Since the columns have the same value for all rows read from
/// each individual file (such as dates), they are typically
/// dictionary encoded for efficiency. You may use
/// [`wrap_partition_type_in_dict`] to request a
/// dictionary-encoded type.
///
/// - The partition columns are solely extracted from the file path. Especially they are NOT part of the parquet files itself.
///
/// # Example
///
/// ```
/// # use std::sync::Arc;
/// # use arrow::datatypes::DataType;
/// # use datafusion::prelude::col;
/// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat};
///
/// // listing options for files with paths such as `/mnt/data/col_a=x/col_b=y/data.parquet`
/// // `col_a` and `col_b` will be included in the data read from those files
/// let listing_options = ListingOptions::new(Arc::new(
/// ParquetFormat::default()
/// ))
/// .with_table_partition_cols(vec![("col_a".to_string(), DataType::Utf8),
/// ("col_b".to_string(), DataType::Utf8)]);
///
/// assert_eq!(listing_options.table_partition_cols, vec![("col_a".to_string(), DataType::Utf8),
/// ("col_b".to_string(), DataType::Utf8)]);
/// ```
///
/// [Hive Partitioning]: https://docs.cloudera.com/HDPDocuments/HDP2/HDP-2.1.3/bk_system-admin-guide/content/hive_partitioned_tables.html
/// [`wrap_partition_type_in_dict`]: crate::datasource::physical_plan::wrap_partition_type_in_dict
pub fn with_table_partition_cols(
mut self,
table_partition_cols: Vec<(String, DataType)>,
) -> Self {
self.table_partition_cols = table_partition_cols;
self
}
/// Set stat collection on [`ListingOptions`] and returns self.
///
/// ```
/// # use std::sync::Arc;
/// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat};
///
/// let listing_options = ListingOptions::new(Arc::new(
/// ParquetFormat::default()
/// ))
/// .with_collect_stat(true);
///
/// assert_eq!(listing_options.collect_stat, true);
/// ```
pub fn with_collect_stat(mut self, collect_stat: bool) -> Self {
self.collect_stat = collect_stat;
self
}
/// Set number of target partitions on [`ListingOptions`] and returns self.
///
/// ```
/// # use std::sync::Arc;
/// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat};
///
/// let listing_options = ListingOptions::new(Arc::new(
/// ParquetFormat::default()
/// ))
/// .with_target_partitions(8);
///
/// assert_eq!(listing_options.target_partitions, 8);
/// ```
pub fn with_target_partitions(mut self, target_partitions: usize) -> Self {
self.target_partitions = target_partitions;
self
}
/// Set file sort order on [`ListingOptions`] and returns self.
///
/// ```
/// # use std::sync::Arc;
/// # use datafusion::prelude::col;
/// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat};
///
/// // Tell datafusion that the files are sorted by column "a"
/// let file_sort_order = vec![vec![
/// col("a").sort(true, true)
/// ]];
///
/// let listing_options = ListingOptions::new(Arc::new(
/// ParquetFormat::default()
/// ))
/// .with_file_sort_order(file_sort_order.clone());
///
/// assert_eq!(listing_options.file_sort_order, file_sort_order);
/// ```
pub fn with_file_sort_order(mut self, file_sort_order: Vec<Vec<Expr>>) -> Self {
self.file_sort_order = file_sort_order;
self
}
/// Infer the schema of the files at the given path on the provided object store.
/// The inferred schema does not include the partitioning columns.
///
/// This method will not be called by the table itself but before creating it.
/// This way when creating the logical plan we can decide to resolve the schema
/// locally or ask a remote service to do it (e.g a scheduler).
pub async fn infer_schema<'a>(
&'a self,
state: &SessionState,
table_path: &'a ListingTableUrl,
) -> Result<SchemaRef> {
let store = state.runtime_env().object_store(table_path)?;
let files: Vec<_> = table_path
.list_all_files(store.as_ref(), &self.file_extension)
.try_collect()
.await?;
self.format.infer_schema(state, &store, &files).await
}
}
/// Collected statistics for files
/// Cache is invalided when file size or last modification has changed
#[derive(Default)]
struct StatisticsCache {
statistics: DashMap<Path, (ObjectMeta, Statistics)>,
}
impl StatisticsCache {
/// Get `Statistics` for file location. Returns None if file has changed or not found.
fn get(&self, meta: &ObjectMeta) -> Option<Statistics> {
self.statistics
.get(&meta.location)
.map(|s| {
let (saved_meta, statistics) = s.value();
if saved_meta.size != meta.size
|| saved_meta.last_modified != meta.last_modified
{
// file has changed
None
} else {
Some(statistics.clone())
}
})
.unwrap_or(None)
}
/// Save collected file statistics
fn save(&self, meta: ObjectMeta, statistics: Statistics) {
self.statistics
.insert(meta.location.clone(), (meta, statistics));
}
}
/// Reads data from one or more files via an
/// [`ObjectStore`](object_store::ObjectStore). For example, from
/// local files or objects from AWS S3. Implements [`TableProvider`],
/// a DataFusion data source.
///
/// # Features
///
/// 1. Merges schemas if the files have compatible but not indentical schemas
///
/// 2. Hive-style partitioning support, where a path such as
/// `/files/date=1/1/2022/data.parquet` is injected as a `date` column.
///
/// 3. Projection pushdown for formats that support it such as such as
/// Parquet
///
/// # Example
///
/// Here is an example of reading a directory of parquet files using a
/// [`ListingTable`]:
///
/// ```no_run
/// # use datafusion::prelude::SessionContext;
/// # use datafusion::error::Result;
/// # use std::sync::Arc;
/// # use datafusion::datasource::{
/// # listing::{
/// # ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl,
/// # },
/// # file_format::parquet::ParquetFormat,
/// # };
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let ctx = SessionContext::new();
/// let session_state = ctx.state();
/// let table_path = "/path/to/parquet";
///
/// // Parse the path
/// let table_path = ListingTableUrl::parse(table_path)?;
///
/// // Create default parquet options
/// let file_format = ParquetFormat::new();
/// let listing_options = ListingOptions::new(Arc::new(file_format))
/// .with_file_extension(".parquet");
///
/// // Resolve the schema
/// let resolved_schema = listing_options
/// .infer_schema(&session_state, &table_path)
/// .await?;
///
/// let config = ListingTableConfig::new(table_path)
/// .with_listing_options(listing_options)
/// .with_schema(resolved_schema);
///
/// // Create a new TableProvider
/// let provider = Arc::new(ListingTable::try_new(config)?);
///
/// // This provider can now be read as a dataframe:
/// let df = ctx.read_table(provider.clone());
///
/// // or registered as a named table:
/// ctx.register_table("my_table", provider);
///
/// # Ok(())
/// # }
/// ```
pub struct ListingTable {
table_paths: Vec<ListingTableUrl>,
/// File fields only
file_schema: SchemaRef,
/// File fields + partition columns
table_schema: SchemaRef,
options: ListingOptions,
definition: Option<String>,
collected_statistics: StatisticsCache,
infinite_source: bool,
}
impl ListingTable {
/// Create new [`ListingTable`] that lists the FS to get the files
/// to scan. See [`ListingTable`] for and example.
///
/// Takes a `ListingTableConfig` as input which requires an `ObjectStore` and `table_path`.
/// `ListingOptions` and `SchemaRef` are optional. If they are not
/// provided the file type is inferred based on the file suffix.
/// If the schema is provided then it must be resolved before creating the table
/// and should contain the fields of the file without the table
/// partitioning columns.
///
pub fn try_new(config: ListingTableConfig) -> Result<Self> {
let file_schema = config
.file_schema
.ok_or_else(|| DataFusionError::Internal("No schema provided.".into()))?;
let options = config.options.ok_or_else(|| {
DataFusionError::Internal("No ListingOptions provided".into())
})?;
// Add the partition columns to the file schema
let mut builder = SchemaBuilder::from(file_schema.fields());
for (part_col_name, part_col_type) in &options.table_partition_cols {
builder.push(Field::new(part_col_name, part_col_type.clone(), false));
}
let infinite_source = options.infinite_source;
let table = Self {
table_paths: config.table_paths,
file_schema,
table_schema: Arc::new(builder.finish()),
options,
definition: None,
collected_statistics: Default::default(),
infinite_source,
};
Ok(table)
}
/// Specify the SQL definition for this table, if any
pub fn with_definition(mut self, defintion: Option<String>) -> Self {
self.definition = defintion;
self
}
/// Get paths ref
pub fn table_paths(&self) -> &Vec<ListingTableUrl> {
&self.table_paths
}
/// Get options ref
pub fn options(&self) -> &ListingOptions {
&self.options
}
/// If file_sort_order is specified, creates the appropriate physical expressions
fn try_create_output_ordering(&self) -> Result<Vec<LexOrdering>> {
let mut all_sort_orders = vec![];
for exprs in &self.options.file_sort_order {
// Construct PhsyicalSortExpr objects from Expr objects:
let sort_exprs = exprs
.iter()
.map(|expr| {
if let Expr::Sort(Sort { expr, asc, nulls_first }) = expr {
if let Expr::Column(col) = expr.as_ref() {
let expr = physical_plan::expressions::col(&col.name, self.table_schema.as_ref())?;
Ok(PhysicalSortExpr {
expr,
options: SortOptions {
descending: !asc,
nulls_first: *nulls_first,
},
})
}
else {
Err(DataFusionError::Plan(
format!("Expected single column references in output_ordering, got {expr}")
))
}
} else {
Err(DataFusionError::Plan(
format!("Expected Expr::Sort in output_ordering, but got {expr}")
))
}
})
.collect::<Result<Vec<_>>>()?;
all_sort_orders.push(sort_exprs);
}
Ok(all_sort_orders)
}
}
#[async_trait]
impl TableProvider for ListingTable {
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
Arc::clone(&self.table_schema)
}
fn table_type(&self) -> TableType {
TableType::Base
}
async fn scan(
&self,
state: &SessionState,
projection: Option<&Vec<usize>>,
filters: &[Expr],
limit: Option<usize>,
) -> Result<Arc<dyn ExecutionPlan>> {
let (partitioned_file_lists, statistics) =
self.list_files_for_scan(state, filters, limit).await?;
// if no files need to be read, return an `EmptyExec`
if partitioned_file_lists.is_empty() {
let schema = self.schema();
let projected_schema = project_schema(&schema, projection)?;
return Ok(Arc::new(EmptyExec::new(false, projected_schema)));
}
// extract types of partition columns
let table_partition_cols = self
.options
.table_partition_cols
.iter()
.map(|col| {
Ok((
col.0.to_owned(),
self.table_schema
.field_with_name(&col.0)?
.data_type()
.clone(),
))
})
.collect::<Result<Vec<_>>>()?;
let filters = if let Some(expr) = conjunction(filters.to_vec()) {
// NOTE: Use the table schema (NOT file schema) here because `expr` may contain references to partition columns.
let table_df_schema = self.table_schema.as_ref().clone().to_dfschema()?;
let filters = create_physical_expr(
&expr,
&table_df_schema,
&self.table_schema,
state.execution_props(),
)?;
Some(filters)
} else {
None
};
let object_store_url = if let Some(url) = self.table_paths.get(0) {
url.object_store()
} else {
return Ok(Arc::new(EmptyExec::new(false, Arc::new(Schema::empty()))));
};
// create the execution plan
self.options
.format
.create_physical_plan(
state,
FileScanConfig {
object_store_url,
file_schema: Arc::clone(&self.file_schema),
file_groups: partitioned_file_lists,
statistics,
projection: projection.cloned(),
limit,
output_ordering: self.try_create_output_ordering()?,
table_partition_cols,
infinite_source: self.infinite_source,
},
filters.as_ref(),
)
.await
}
fn supports_filter_pushdown(
&self,
filter: &Expr,
) -> Result<TableProviderFilterPushDown> {
if expr_applicable_for_cols(
&self
.options
.table_partition_cols
.iter()
.map(|x| x.0.clone())
.collect::<Vec<_>>(),
filter,
) {
// if filter can be handled by partiton pruning, it is exact
Ok(TableProviderFilterPushDown::Exact)
} else {
// otherwise, we still might be able to handle the filter with file
// level mechanisms such as Parquet row group pruning.
Ok(TableProviderFilterPushDown::Inexact)
}
}
fn get_table_definition(&self) -> Option<&str> {
self.definition.as_deref()
}
async fn insert_into(
&self,
state: &SessionState,
input: Arc<dyn ExecutionPlan>,
) -> Result<Arc<dyn ExecutionPlan>> {
// Check that the schema of the plan matches the schema of this table.
if !input.schema().eq(&self.schema()) {
return Err(DataFusionError::Plan(
// Return an error if schema of the input query does not match with the table schema.
"Inserting query must have the same schema with the table.".to_string(),
));
}
if self.table_paths().len() > 1 {
return Err(DataFusionError::Plan(
"Writing to a table backed by multiple files is not supported yet"
.to_owned(),
));
}
let table_path = &self.table_paths()[0];
// Get the object store for the table path.
let store = state.runtime_env().object_store(table_path)?;
let file_list_stream = pruned_partition_list(
store.as_ref(),
table_path,
&[],
&self.options.file_extension,
&self.options.table_partition_cols,
)
.await?;
let file_groups = file_list_stream.try_collect::<Vec<_>>().await?;
if file_groups.len() > 1 {
return Err(DataFusionError::Plan(
"Datafusion currently supports tables from single partition and/or file."
.to_owned(),
));
}
// Sink related option, apart from format
let config = FileSinkConfig {
object_store_url: self.table_paths()[0].object_store(),
file_groups,
output_schema: input.schema(),
table_partition_cols: self.options.table_partition_cols.clone(),
writer_mode: crate::datasource::file_format::FileWriterMode::Append,
};
self.options()
.format
.create_writer_physical_plan(input, state, config)
.await
}
}
impl ListingTable {
/// Get the list of files for a scan as well as the file level statistics.
/// The list is grouped to let the execution plan know how the files should
/// be distributed to different threads / executors.
async fn list_files_for_scan<'a>(
&'a self,
ctx: &'a SessionState,
filters: &'a [Expr],
limit: Option<usize>,
) -> Result<(Vec<Vec<PartitionedFile>>, Statistics)> {
let store = if let Some(url) = self.table_paths.get(0) {
ctx.runtime_env().object_store(url)?
} else {
return Ok((vec![], Statistics::default()));
};
// list files (with partitions)
let file_list = future::try_join_all(self.table_paths.iter().map(|table_path| {
pruned_partition_list(
store.as_ref(),
table_path,
filters,
&self.options.file_extension,
&self.options.table_partition_cols,
)
}))
.await?;
let file_list = stream::iter(file_list).flatten();
// collect the statistics if required by the config
let files = file_list.then(|part_file| async {
let part_file = part_file?;
let statistics = if self.options.collect_stat {
match self.collected_statistics.get(&part_file.object_meta) {
Some(statistics) => statistics,
None => {
let statistics = self
.options
.format
.infer_stats(
ctx,
&store,
self.file_schema.clone(),
&part_file.object_meta,
)
.await?;
self.collected_statistics
.save(part_file.object_meta.clone(), statistics.clone());
statistics
}
}
} else {
Statistics::default()
};
Ok((part_file, statistics)) as Result<(PartitionedFile, Statistics)>
});
let (files, statistics) =
get_statistics_with_limit(files, self.schema(), limit).await?;
Ok((
split_files(files, self.options.target_partitions),
statistics,
))
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::datasource::file_format::file_type::GetExt;
use crate::datasource::{provider_as_source, MemTable};
use crate::physical_plan::collect;
use crate::prelude::*;
use crate::{
assert_batches_eq,
datasource::file_format::{avro::AvroFormat, parquet::ParquetFormat},
execution::options::ReadOptions,
logical_expr::{col, lit},
test::{columns, object_store::register_test_store},
};
use arrow::csv;
use arrow::datatypes::{DataType, Schema};
use arrow::error::Result as ArrowResult;
use arrow::record_batch::RecordBatch;
use chrono::DateTime;
use datafusion_common::assert_contains;
use datafusion_expr::LogicalPlanBuilder;
use rstest::*;
use std::fs::File;
use tempfile::TempDir;
/// It creates dummy file and checks if it can create unbounded input executors.
async fn unbounded_table_helper(
file_type: FileType,
listing_option: ListingOptions,
infinite_data: bool,
) -> Result<()> {
let ctx = SessionContext::new();
register_test_store(
&ctx,
&[(&format!("table/file{}", file_type.get_ext()), 100)],
);
let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);
let table_path = ListingTableUrl::parse("test:///table/").unwrap();
let config = ListingTableConfig::new(table_path)
.with_listing_options(listing_option)
.with_schema(Arc::new(schema));
// Create a table
let table = ListingTable::try_new(config)?;
// Create executor from table
let source_exec = table.scan(&ctx.state(), None, &[], None).await?;
assert_eq!(source_exec.unbounded_output(&[])?, infinite_data);
Ok(())
}
#[tokio::test]
async fn read_single_file() -> Result<()> {
let ctx = SessionContext::new();
let table = load_table(&ctx, "alltypes_plain.parquet").await?;
let projection = None;
let exec = table
.scan(&ctx.state(), projection, &[], None)
.await
.expect("Scan table");
assert_eq!(exec.children().len(), 0);
assert_eq!(exec.output_partitioning().partition_count(), 1);
// test metadata
assert_eq!(exec.statistics().num_rows, Some(8));
assert_eq!(exec.statistics().total_byte_size, Some(671));
Ok(())
}
#[tokio::test]
async fn load_table_stats_by_default() -> Result<()> {
let testdata = crate::test_util::parquet_test_data();
let filename = format!("{}/{}", testdata, "alltypes_plain.parquet");
let table_path = ListingTableUrl::parse(filename).unwrap();
let ctx = SessionContext::new();
let state = ctx.state();
let opt = ListingOptions::new(Arc::new(ParquetFormat::default()));
let schema = opt.infer_schema(&state, &table_path).await?;
let config = ListingTableConfig::new(table_path)
.with_listing_options(opt)
.with_schema(schema);
let table = ListingTable::try_new(config)?;
let exec = table.scan(&state, None, &[], None).await?;
assert_eq!(exec.statistics().num_rows, Some(8));
assert_eq!(exec.statistics().total_byte_size, Some(671));
Ok(())
}
#[tokio::test]
async fn load_table_stats_when_no_stats() -> Result<()> {
let testdata = crate::test_util::parquet_test_data();
let filename = format!("{}/{}", testdata, "alltypes_plain.parquet");
let table_path = ListingTableUrl::parse(filename).unwrap();
let ctx = SessionContext::new();
let state = ctx.state();
let opt = ListingOptions::new(Arc::new(ParquetFormat::default()))
.with_collect_stat(false);
let schema = opt.infer_schema(&state, &table_path).await?;
let config = ListingTableConfig::new(table_path)
.with_listing_options(opt)
.with_schema(schema);
let table = ListingTable::try_new(config)?;
let exec = table.scan(&state, None, &[], None).await?;
assert_eq!(exec.statistics().num_rows, None);
assert_eq!(exec.statistics().total_byte_size, None);
Ok(())
}
#[tokio::test]
async fn test_try_create_output_ordering() {
let testdata = crate::test_util::parquet_test_data();
let filename = format!("{}/{}", testdata, "alltypes_plain.parquet");
let table_path = ListingTableUrl::parse(filename).unwrap();
let ctx = SessionContext::new();
let state = ctx.state();
let options = ListingOptions::new(Arc::new(ParquetFormat::default()));
let schema = options.infer_schema(&state, &table_path).await.unwrap();
use crate::physical_plan::expressions::col as physical_col;
use std::ops::Add;
// (file_sort_order, expected_result)
let cases = vec![
(vec![], Ok(vec![])),
// not a sort expr
(
vec![vec![col("string_col")]],
Err("Expected Expr::Sort in output_ordering, but got string_col"),
),
// sort expr, but non column
(
vec![vec![
col("int_col").add(lit(1)).sort(true, true),
]],
Err("Expected single column references in output_ordering, got int_col + Int32(1)"),
),
// ok with one column
(
vec![vec![col("string_col").sort(true, false)]],
Ok(vec![vec![PhysicalSortExpr {
expr: physical_col("string_col", &schema).unwrap(),
options: SortOptions {
descending: false,
nulls_first: false,
},
}]])
),
// ok with two columns, different options
(
vec![vec![
col("string_col").sort(true, false),
col("int_col").sort(false, true),
]],
Ok(vec![vec![
PhysicalSortExpr {
expr: physical_col("string_col", &schema).unwrap(),
options: SortOptions {
descending: false,
nulls_first: false,
},
},
PhysicalSortExpr {
expr: physical_col("int_col", &schema).unwrap(),
options: SortOptions {
descending: true,
nulls_first: true,
},
},
]])
),
];
for (file_sort_order, expected_result) in cases {
let options = options.clone().with_file_sort_order(file_sort_order);
let config = ListingTableConfig::new(table_path.clone())
.with_listing_options(options)
.with_schema(schema.clone());
let table =
ListingTable::try_new(config.clone()).expect("Creating the table");
let ordering_result = table.try_create_output_ordering();
match (expected_result, ordering_result) {
(Ok(expected), Ok(result)) => {
assert_eq!(expected, result);
}
(Err(expected), Err(result)) => {
// can't compare the DataFusionError directly
let result = result.to_string();
let expected = expected.to_string();
assert_contains!(result.to_string(), expected);
}
(expected_result, ordering_result) => {
panic!(
"expected: {expected_result:#?}\n\nactual:{ordering_result:#?}"
);
}
}
}
}
#[tokio::test]
async fn read_empty_table() -> Result<()> {
let ctx = SessionContext::new();
let path = String::from("table/p1=v1/file.avro");
register_test_store(&ctx, &[(&path, 100)]);
let opt = ListingOptions::new(Arc::new(AvroFormat {}))
.with_file_extension(FileType::AVRO.get_ext())
.with_table_partition_cols(vec![(String::from("p1"), DataType::Utf8)])
.with_target_partitions(4);
let table_path = ListingTableUrl::parse("test:///table/").unwrap();
let file_schema =
Arc::new(Schema::new(vec![Field::new("a", DataType::Boolean, false)]));
let config = ListingTableConfig::new(table_path)
.with_listing_options(opt)
.with_schema(file_schema);
let table = ListingTable::try_new(config)?;
assert_eq!(
columns(&table.schema()),
vec!["a".to_owned(), "p1".to_owned()]
);
// this will filter out the only file in the store
let filter = Expr::not_eq(col("p1"), lit("v1"));
let scan = table
.scan(&ctx.state(), None, &[filter], None)
.await
.expect("Empty execution plan");
assert!(scan.as_any().is::<EmptyExec>());
assert_eq!(
columns(&scan.schema()),
vec!["a".to_owned(), "p1".to_owned()]
);
Ok(())
}
#[tokio::test]
async fn unbounded_csv_table_without_schema() -> Result<()> {
let tmp_dir = TempDir::new()?;
let file_path = tmp_dir.path().join("dummy.csv");
File::create(file_path)?;
let ctx = SessionContext::new();
let error = ctx
.register_csv(
"test",
tmp_dir.path().to_str().unwrap(),
CsvReadOptions::new().mark_infinite(true),
)
.await
.unwrap_err();
match error {
DataFusionError::Plan(_) => Ok(()),
val => Err(val),
}
}
#[tokio::test]
async fn unbounded_json_table_without_schema() -> Result<()> {
let tmp_dir = TempDir::new()?;
let file_path = tmp_dir.path().join("dummy.json");
File::create(file_path)?;
let ctx = SessionContext::new();
let error = ctx
.register_json(
"test",
tmp_dir.path().to_str().unwrap(),
NdJsonReadOptions::default().mark_infinite(true),
)
.await
.unwrap_err();
match error {
DataFusionError::Plan(_) => Ok(()),
val => Err(val),
}
}
#[tokio::test]
async fn unbounded_avro_table_without_schema() -> Result<()> {
let tmp_dir = TempDir::new()?;
let file_path = tmp_dir.path().join("dummy.avro");
File::create(file_path)?;
let ctx = SessionContext::new();
let error = ctx
.register_avro(
"test",
tmp_dir.path().to_str().unwrap(),
AvroReadOptions::default().mark_infinite(true),
)
.await
.unwrap_err();
match error {
DataFusionError::Plan(_) => Ok(()),
val => Err(val),
}
}
#[rstest]
#[tokio::test]
async fn unbounded_csv_table(
#[values(true, false)] infinite_data: bool,
) -> Result<()> {
let config = CsvReadOptions::new().mark_infinite(infinite_data);
let session_config = SessionConfig::new().with_target_partitions(1);
let listing_options = config.to_listing_options(&session_config);
unbounded_table_helper(FileType::CSV, listing_options, infinite_data).await
}
#[rstest]
#[tokio::test]
async fn unbounded_json_table(
#[values(true, false)] infinite_data: bool,
) -> Result<()> {
let config = NdJsonReadOptions::default().mark_infinite(infinite_data);
let session_config = SessionConfig::new().with_target_partitions(1);
let listing_options = config.to_listing_options(&session_config);
unbounded_table_helper(FileType::JSON, listing_options, infinite_data).await
}
#[rstest]
#[tokio::test]
async fn unbounded_avro_table(
#[values(true, false)] infinite_data: bool,
) -> Result<()> {
let config = AvroReadOptions::default().mark_infinite(infinite_data);
let session_config = SessionConfig::new().with_target_partitions(1);
let listing_options = config.to_listing_options(&session_config);
unbounded_table_helper(FileType::AVRO, listing_options, infinite_data).await
}
#[tokio::test]
async fn test_assert_list_files_for_scan_grouping() -> Result<()> {
// more expected partitions than files
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0",
"bucket/key-prefix/file1",
"bucket/key-prefix/file2",
"bucket/key-prefix/file3",
"bucket/key-prefix/file4",
],
"test:///bucket/key-prefix/",
12,
5,
)
.await?;
// as many expected partitions as files
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0",
"bucket/key-prefix/file1",
"bucket/key-prefix/file2",
"bucket/key-prefix/file3",
],
"test:///bucket/key-prefix/",
4,
4,
)
.await?;
// more files as expected partitions
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0",
"bucket/key-prefix/file1",
"bucket/key-prefix/file2",
"bucket/key-prefix/file3",
"bucket/key-prefix/file4",
],
"test:///bucket/key-prefix/",
2,
2,
)
.await?;
// no files => no groups
assert_list_files_for_scan_grouping(&[], "test:///bucket/key-prefix/", 2, 0)
.await?;
// files that don't match the prefix
assert_list_files_for_scan_grouping(
&[
"bucket/key-prefix/file0",
"bucket/key-prefix/file1",
"bucket/other-prefix/roguefile",
],
"test:///bucket/key-prefix/",
10,
2,
)
.await?;
Ok(())
}
#[tokio::test]
async fn test_assert_list_files_for_multi_path() -> Result<()> {
// more expected partitions than files
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0",
"bucket/key1/file1",
"bucket/key1/file2",
"bucket/key2/file3",
"bucket/key2/file4",
"bucket/key3/file5",
],
&["test:///bucket/key1/", "test:///bucket/key2/"],
12,
5,
)
.await?;
// as many expected partitions as files
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0",
"bucket/key1/file1",
"bucket/key1/file2",
"bucket/key2/file3",
"bucket/key2/file4",
"bucket/key3/file5",
],
&["test:///bucket/key1/", "test:///bucket/key2/"],
5,
5,
)
.await?;
// more files as expected partitions
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0",
"bucket/key1/file1",
"bucket/key1/file2",
"bucket/key2/file3",
"bucket/key2/file4",
"bucket/key3/file5",
],
&["test:///bucket/key1/"],
2,
2,
)
.await?;
// no files => no groups
assert_list_files_for_multi_paths(&[], &["test:///bucket/key1/"], 2, 0).await?;
// files that don't match the prefix
assert_list_files_for_multi_paths(
&[
"bucket/key1/file0",
"bucket/key1/file1",
"bucket/key1/file2",
"bucket/key2/file3",
"bucket/key2/file4",
"bucket/key3/file5",
],
&["test:///bucket/key3/"],
2,
1,
)
.await?;
Ok(())
}
async fn load_table(
ctx: &SessionContext,
name: &str,
) -> Result<Arc<dyn TableProvider>> {
let testdata = crate::test_util::parquet_test_data();
let filename = format!("{testdata}/{name}");
let table_path = ListingTableUrl::parse(filename).unwrap();
let config = ListingTableConfig::new(table_path)
.infer(&ctx.state())
.await?;
let table = ListingTable::try_new(config)?;
Ok(Arc::new(table))
}
fn load_empty_schema_csv_table(
schema: SchemaRef,
temp_path: &str,
) -> Result<Arc<dyn TableProvider>> {
File::create(temp_path)?;
let table_path = ListingTableUrl::parse(temp_path).unwrap();
let file_format = CsvFormat::default();
let listing_options = ListingOptions::new(Arc::new(file_format));
let config = ListingTableConfig::new(table_path)
.with_listing_options(listing_options)
.with_schema(schema);
let table = ListingTable::try_new(config)?;
Ok(Arc::new(table))
}
/// Check that the files listed by the table match the specified `output_partitioning`
/// when the object store contains `files`.
async fn assert_list_files_for_scan_grouping(
files: &[&str],
table_prefix: &str,
target_partitions: usize,
output_partitioning: usize,
) -> Result<()> {
let ctx = SessionContext::new();
register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::<Vec<_>>());
let format = AvroFormat {};
let opt = ListingOptions::new(Arc::new(format))
.with_file_extension("")
.with_target_partitions(target_partitions);
let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);
let table_path = ListingTableUrl::parse(table_prefix).unwrap();
let config = ListingTableConfig::new(table_path)
.with_listing_options(opt)
.with_schema(Arc::new(schema));
let table = ListingTable::try_new(config)?;
let (file_list, _) = table.list_files_for_scan(&ctx.state(), &[], None).await?;
assert_eq!(file_list.len(), output_partitioning);
Ok(())
}
/// Check that the files listed by the table match the specified `output_partitioning`
/// when the object store contains `files`.
async fn assert_list_files_for_multi_paths(
files: &[&str],
table_prefix: &[&str],
target_partitions: usize,
output_partitioning: usize,
) -> Result<()> {
let ctx = SessionContext::new();
register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::<Vec<_>>());
let format = AvroFormat {};
let opt = ListingOptions::new(Arc::new(format))
.with_file_extension("")
.with_target_partitions(target_partitions);
let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);
let table_paths = table_prefix
.iter()
.map(|t| ListingTableUrl::parse(t).unwrap())
.collect();
let config = ListingTableConfig::new_with_multi_paths(table_paths)
.with_listing_options(opt)
.with_schema(Arc::new(schema));
let table = ListingTable::try_new(config)?;
let (file_list, _) = table.list_files_for_scan(&ctx.state(), &[], None).await?;
assert_eq!(file_list.len(), output_partitioning);
Ok(())
}
#[test]
fn test_statistics_cache() {
let meta = ObjectMeta {
location: Path::from("test"),
last_modified: DateTime::parse_from_rfc3339("2022-09-27T22:36:00+02:00")
.unwrap()
.into(),
size: 1024,
e_tag: None,
};
let cache = StatisticsCache::default();
assert!(cache.get(&meta).is_none());
cache.save(meta.clone(), Statistics::default());
assert!(cache.get(&meta).is_some());
// file size changed
let mut meta2 = meta.clone();
meta2.size = 2048;
assert!(cache.get(&meta2).is_none());
// file last_modified changed
let mut meta2 = meta.clone();
meta2.last_modified = DateTime::parse_from_rfc3339("2022-09-27T22:40:00+02:00")
.unwrap()
.into();
assert!(cache.get(&meta2).is_none());
// different file
let mut meta2 = meta;
meta2.location = Path::from("test2");
assert!(cache.get(&meta2).is_none());
}
#[tokio::test]
async fn test_append_plan_to_external_table_stored_as_csv() -> Result<()> {
let file_type = FileType::CSV;
let file_compression_type = FileCompressionType::UNCOMPRESSED;
// Create the initial context, schema, and batch.
let session_ctx = SessionContext::new();
// Create a new schema with one field called "a" of type Int32
let schema = Arc::new(Schema::new(vec![Field::new(
"column1",
DataType::Int32,
false,
)]));
// Create a new batch of data to insert into the table
let batch = RecordBatch::try_new(
schema.clone(),
vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))],
)?;
// Filename with extension
let filename = format!(
"path{}",
file_type
.to_owned()
.get_ext_with_compression(file_compression_type.clone())
.unwrap()
);
// Define batch size for file reader
let batch_size = batch.num_rows();
// Create a temporary directory and a CSV file within it.
let tmp_dir = TempDir::new()?;
let path = tmp_dir.path().join(filename);
let initial_table =
load_empty_schema_csv_table(schema.clone(), path.to_str().unwrap())?;
session_ctx.register_table("t", initial_table)?;
// Create and register the source table with the provided schema and inserted data
let source_table = Arc::new(MemTable::try_new(
schema.clone(),
vec![vec![batch.clone(), batch.clone()]],
)?);
session_ctx.register_table("source", source_table.clone())?;
// Convert the source table into a provider so that it can be used in a query
let source = provider_as_source(source_table);
// Create a table scan logical plan to read from the source table
let scan_plan = LogicalPlanBuilder::scan("source", source, None)?.build()?;
// Create an insert plan to insert the source data into the initial table
let insert_into_table =
LogicalPlanBuilder::insert_into(scan_plan, "t", &schema)?.build()?;
// Create a physical plan from the insert plan
let plan = session_ctx
.state()
.create_physical_plan(&insert_into_table)
.await?;
// Execute the physical plan and collect the results
let res = collect(plan, session_ctx.task_ctx()).await?;
// Insert returns the number of rows written, in our case this would be 6.
let expected = vec![
"+-------+",
"| count |",
"+-------+",
"| 6 |",
"+-------+",
];
// Assert that the batches read from the file match the expected result.
assert_batches_eq!(expected, &res);
// Open the CSV file, read its contents as a record batch, and collect the batches into a vector.
let file = File::open(path.clone())?;
let reader = csv::ReaderBuilder::new(schema.clone())
.has_header(true)
.with_batch_size(batch_size)
.build(file)
.map_err(|e| DataFusionError::Internal(e.to_string()))?;
let batches = reader
.collect::<Vec<ArrowResult<RecordBatch>>>()
.into_iter()
.collect::<ArrowResult<Vec<RecordBatch>>>()
.map_err(|e| DataFusionError::Internal(e.to_string()))?;
// Define the expected result as a vector of strings.
let expected = vec![
"+---------+",
"| column1 |",
"+---------+",
"| 1 |",
"| 2 |",
"| 3 |",
"| 1 |",
"| 2 |",
"| 3 |",
"+---------+",
];
// Assert that the batches read from the file match the expected result.
assert_batches_eq!(expected, &batches);
// Create a physical plan from the insert plan
let plan = session_ctx
.state()
.create_physical_plan(&insert_into_table)
.await?;
// Again, execute the physical plan and collect the results
let res = collect(plan, session_ctx.task_ctx()).await?;
// Insert returns the number of rows written, in our case this would be 6.
let expected = vec![
"+-------+",
"| count |",
"+-------+",
"| 6 |",
"+-------+",
];
// Assert that the batches read from the file match the expected result.
assert_batches_eq!(expected, &res);
// Open the CSV file, read its contents as a record batch, and collect the batches into a vector.
let file = File::open(path.clone())?;
let reader = csv::ReaderBuilder::new(schema.clone())
.has_header(true)
.with_batch_size(batch_size)
.build(file)
.map_err(|e| DataFusionError::Internal(e.to_string()))?;
let batches = reader
.collect::<Vec<ArrowResult<RecordBatch>>>()
.into_iter()
.collect::<ArrowResult<Vec<RecordBatch>>>()
.map_err(|e| DataFusionError::Internal(e.to_string()));
// Define the expected result after the second append.
let expected = vec![
"+---------+",
"| column1 |",
"+---------+",
"| 1 |",
"| 2 |",
"| 3 |",
"| 1 |",
"| 2 |",
"| 3 |",
"| 1 |",
"| 2 |",
"| 3 |",
"| 1 |",
"| 2 |",
"| 3 |",
"+---------+",
];
// Assert that the batches read from the file after the second append match the expected result.
assert_batches_eq!(expected, &batches?);
// Return Ok if the function
Ok(())
}
}