From c184efa9f27eae9d1ba3cb5e3a1f139de3f3bb7c Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 01/17] [feat] adding partition_values to PartitionedFile --- ballista/rust/core/proto/ballista.proto | 1 + .../src/serde/physical_plan/from_proto.rs | 38 ++++++++++------ .../core/src/serde/physical_plan/to_proto.rs | 42 +++++++++++------- datafusion/src/datasource/file_format/avro.rs | 13 ++---- datafusion/src/datasource/file_format/csv.rs | 9 ++-- datafusion/src/datasource/file_format/json.rs | 9 ++-- .../src/datasource/file_format/parquet.rs | 13 ++---- datafusion/src/datasource/listing.rs | 43 +++++++------------ datafusion/src/datasource/mod.rs | 6 ++- .../src/datasource/object_store/local.rs | 18 +++++--- .../src/physical_plan/file_format/avro.rs | 6 +-- .../src/physical_plan/file_format/csv.rs | 10 ++--- .../src/physical_plan/file_format/json.rs | 10 ++--- .../src/physical_plan/file_format/parquet.rs | 6 +-- datafusion/src/test/mod.rs | 8 +--- 15 files changed, 110 insertions(+), 122 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 95b78fcc6d24..ce3bdb7cd95b 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -274,6 +274,7 @@ message PartitionedFile { string path = 1; uint64 size = 2; uint64 last_modified_ns = 3; + repeated ScalarValue partition_values = 4; } message CsvFormat { diff --git a/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/ballista/rust/core/src/serde/physical_plan/from_proto.rs index dce354ac69fa..5864b6155282 100644 --- a/ballista/rust/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -127,8 +127,8 @@ impl TryInto> for &protobuf::PhysicalPlanNode { Arc::new(LocalFileSystem {}), scan.file_groups .iter() - .map(|p| p.into()) - .collect::>>(), + .map(|f| f.try_into()) + .collect::, _>>()?, statistics, schema, scan.has_header, @@ -147,8 +147,8 @@ impl TryInto> for &protobuf::PhysicalPlanNode { Arc::new(LocalFileSystem {}), scan.file_groups .iter() - .map(|p| p.into()) - .collect::>>(), + .map(|p| p.try_into()) + .collect::, _>>()?, statistics, schema, Some(projection), @@ -167,8 +167,8 @@ impl TryInto> for &protobuf::PhysicalPlanNode { Arc::new(LocalFileSystem {}), scan.file_groups .iter() - .map(|p| p.into()) - .collect::>>(), + .map(|f| f.try_into()) + .collect::, _>>()?, statistics, schema, Some(projection), @@ -738,9 +738,11 @@ pub fn parse_protobuf_hash_partitioning( } } -impl From<&protobuf::PartitionedFile> for PartitionedFile { - fn from(val: &protobuf::PartitionedFile) -> Self { - PartitionedFile { +impl TryFrom<&protobuf::PartitionedFile> for PartitionedFile { + type Error = BallistaError; + + fn try_from(val: &protobuf::PartitionedFile) -> Result { + Ok(PartitionedFile { file_meta: FileMeta { sized_file: SizedFile { path: val.path.clone(), @@ -752,13 +754,23 @@ impl From<&protobuf::PartitionedFile> for PartitionedFile { Some(Utc.timestamp_nanos(val.last_modified_ns as i64)) }, }, - } + partition_values: val + .partition_values + .iter() + .map(|v| v.try_into()) + .collect::, _>>()?, + }) } } -impl From<&protobuf::FileGroup> for Vec { - fn from(val: &protobuf::FileGroup) -> Self { - val.files.iter().map(|f| f.into()).collect() +impl TryFrom<&protobuf::FileGroup> for Vec { + type Error = BallistaError; + + fn try_from(val: &protobuf::FileGroup) -> Result { + val.files + .iter() + .map(|f| f.try_into()) + .collect::, _>>() } } diff --git a/ballista/rust/core/src/serde/physical_plan/to_proto.rs b/ballista/rust/core/src/serde/physical_plan/to_proto.rs index 52285eea0a9c..3e65b6484cf6 100644 --- a/ballista/rust/core/src/serde/physical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/to_proto.rs @@ -247,8 +247,8 @@ impl TryInto for Arc { let file_groups = exec .file_groups() .iter() - .map(|p| p.as_slice().into()) - .collect(); + .map(|p| p.as_slice().try_into()) + .collect::, _>>()?; Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::CsvScan( protobuf::CsvScanExecNode { @@ -279,8 +279,8 @@ impl TryInto for Arc { let file_groups = exec .file_groups() .iter() - .map(|p| p.as_slice().into()) - .collect(); + .map(|p| p.as_slice().try_into()) + .collect::, _>>()?; Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::ParquetScan( @@ -305,8 +305,8 @@ impl TryInto for Arc { let file_groups = exec .file_groups() .iter() - .map(|p| p.as_slice().into()) - .collect(); + .map(|p| p.as_slice().try_into()) + .collect::, _>>()?; Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::AvroScan( protobuf::AvroScanExecNode { @@ -674,9 +674,11 @@ fn try_parse_when_then_expr( }) } -impl From<&PartitionedFile> for protobuf::PartitionedFile { - fn from(pf: &PartitionedFile) -> protobuf::PartitionedFile { - protobuf::PartitionedFile { +impl TryFrom<&PartitionedFile> for protobuf::PartitionedFile { + type Error = BallistaError; + + fn try_from(pf: &PartitionedFile) -> Result { + Ok(protobuf::PartitionedFile { path: pf.file_meta.path().to_owned(), size: pf.file_meta.size(), last_modified_ns: pf @@ -684,15 +686,25 @@ impl From<&PartitionedFile> for protobuf::PartitionedFile { .last_modified .map(|ts| ts.timestamp_nanos() as u64) .unwrap_or(0), - } + partition_values: pf + .partition_values + .iter() + .map(|v| v.try_into()) + .collect::, _>>()?, + }) } } -impl From<&[PartitionedFile]> for protobuf::FileGroup { - fn from(gr: &[PartitionedFile]) -> protobuf::FileGroup { - protobuf::FileGroup { - files: gr.iter().map(|f| f.into()).collect(), - } +impl TryFrom<&[PartitionedFile]> for protobuf::FileGroup { + type Error = BallistaError; + + fn try_from(gr: &[PartitionedFile]) -> Result { + Ok(protobuf::FileGroup { + files: gr + .iter() + .map(|f| f.try_into()) + .collect::, _>>()?, + }) } } diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index c6326962e34a..8a90221c4d91 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -79,12 +79,9 @@ impl FileFormat for AvroFormat { #[cfg(feature = "avro")] mod tests { use crate::{ - datasource::{ - object_store::local::{ - local_file_meta, local_object_reader, local_object_reader_stream, - LocalFileSystem, - }, - PartitionedFile, + datasource::object_store::local::{ + local_object_reader, local_object_reader_stream, local_unpartitioned_file, + LocalFileSystem, }, physical_plan::collect, }; @@ -357,9 +354,7 @@ mod tests { .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { - file_meta: local_file_meta(filename.to_owned()), - }]]; + let files = vec![vec![local_unpartitioned_file(filename.to_owned())]]; let exec = format .create_physical_plan(PhysicalPlanConfig { object_store: Arc::new(LocalFileSystem {}), diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index f9959943a2e4..6e2654852046 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -148,10 +148,9 @@ mod tests { datasource::{ file_format::PhysicalPlanConfig, object_store::local::{ - local_file_meta, local_object_reader, local_object_reader_stream, - LocalFileSystem, + local_object_reader, local_object_reader_stream, + local_unpartitioned_file, LocalFileSystem, }, - PartitionedFile, }, physical_plan::collect, }; @@ -269,9 +268,7 @@ mod tests { .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { - file_meta: local_file_meta(filename.to_owned()), - }]]; + let files = vec![vec![local_unpartitioned_file(filename.to_owned())]]; let exec = format .create_physical_plan(PhysicalPlanConfig { object_store: Arc::new(LocalFileSystem {}), diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index a579831c7241..c0ce978d51de 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -116,10 +116,9 @@ mod tests { datasource::{ file_format::PhysicalPlanConfig, object_store::local::{ - local_file_meta, local_object_reader, local_object_reader_stream, - LocalFileSystem, + local_object_reader, local_object_reader_stream, + local_unpartitioned_file, LocalFileSystem, }, - PartitionedFile, }, physical_plan::collect, }; @@ -220,9 +219,7 @@ mod tests { .infer_stats(local_object_reader(filename.to_owned())) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { - file_meta: local_file_meta(filename.to_owned()), - }]]; + let files = vec![vec![local_unpartitioned_file(filename.to_owned())]]; let exec = format .create_physical_plan(PhysicalPlanConfig { object_store: Arc::new(LocalFileSystem {}), diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index 424a2985a3f7..8909fb93a7db 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -330,12 +330,9 @@ impl ChunkReader for ChunkObjectReader { #[cfg(test)] mod tests { use crate::{ - datasource::{ - object_store::local::{ - local_file_meta, local_object_reader, local_object_reader_stream, - LocalFileSystem, - }, - PartitionedFile, + datasource::object_store::local::{ + local_object_reader, local_object_reader_stream, local_unpartitioned_file, + LocalFileSystem, }, physical_plan::collect, }; @@ -603,9 +600,7 @@ mod tests { .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![PartitionedFile { - file_meta: local_file_meta(filename.clone()), - }]]; + let files = vec![vec![local_unpartitioned_file(filename.clone())]]; let exec = format .create_physical_plan(PhysicalPlanConfig { object_store: Arc::new(LocalFileSystem {}), diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index 4af82d0c185a..cd00f2c7069b 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -162,15 +162,8 @@ impl TableProvider for ListingTable { filters: &[Expr], limit: Option, ) -> Result> { - // TODO object_store_registry should be provided as param here - let (partitioned_file_lists, statistics) = self - .list_files_for_scan( - Arc::clone(&self.object_store), - &self.path, - filters, - limit, - ) - .await?; + let (partitioned_file_lists, statistics) = + self.list_files_for_scan(filters, limit).await?; // create the execution plan self.options .format @@ -196,17 +189,17 @@ impl TableProvider for ListingTable { } impl ListingTable { + /// Get the list of files for a scan. 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, - object_store: Arc, - path: &'a str, filters: &'a [Expr], limit: Option, ) -> Result<(Vec>, Statistics)> { // list files (with partitions) let file_list = pruned_partition_list( - object_store.as_ref(), - path, + self.object_store.as_ref(), + &self.path, filters, &self.options.file_extension, &self.options.partitions, @@ -214,6 +207,7 @@ impl ListingTable { .await?; // collect the statistics if required by the config + let object_store = Arc::clone(&self.object_store); let files = file_list.then(move |part_file| { let object_store = object_store.clone(); async move { @@ -260,7 +254,12 @@ async fn pruned_partition_list( store .list_file_with_suffix(path, file_extension) .await? - .map(|f| Ok(PartitionedFile { file_meta: f? })), + .map(|f| { + Ok(PartitionedFile { + file_meta: f?, + partition_values: vec![], + }) + }), )) } else { todo!("use filters to prune partitions") @@ -286,7 +285,7 @@ mod tests { use crate::{ datasource::{ file_format::{avro::AvroFormat, parquet::ParquetFormat}, - object_store::{local::LocalFileSystem, FileMeta, ObjectStore, SizedFile}, + object_store::{local::LocalFileSystem, ObjectStore}, }, test::object_store::TestObjectStore, }; @@ -295,15 +294,7 @@ mod tests { #[test] fn test_split_files() { - let new_partitioned_file = |path: &str| PartitionedFile { - file_meta: FileMeta { - sized_file: SizedFile { - path: path.to_owned(), - size: 10, - }, - last_modified: None, - }, - }; + let new_partitioned_file = |path: &str| PartitionedFile::new(path.to_owned(), 10); let files = vec![ new_partitioned_file("a"), new_partitioned_file("b"), @@ -473,9 +464,7 @@ mod tests { opt, ); - let (file_list, _) = table - .list_files_for_scan(mock_store, table_prefix, &[], None) - .await?; + let (file_list, _) = table.list_files_for_scan(&[], None).await?; assert_eq!(file_list.len(), output_partitioning); diff --git a/datafusion/src/datasource/mod.rs b/datafusion/src/datasource/mod.rs index 2e5330f16cb7..6097a21a52c1 100644 --- a/datafusion/src/datasource/mod.rs +++ b/datafusion/src/datasource/mod.rs @@ -33,6 +33,7 @@ use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; +use crate::scalar::ScalarValue; use futures::StreamExt; use std::pin::Pin; @@ -128,8 +129,8 @@ pub async fn get_statistics_with_limit( pub struct PartitionedFile { /// Path for the file (e.g. URL, filesystem path, etc) pub file_meta: FileMeta, - // Values of partition columns to be appended to each row - // pub partition_value: Option>, + /// Values of partition columns to be appended to each row + pub partition_values: Vec, // We may include row group range here for a more fine-grained parallel execution } @@ -141,6 +142,7 @@ impl PartitionedFile { sized_file: SizedFile { path, size }, last_modified: None, }, + partition_values: vec![], } } } diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs index 4f4dbefbca49..b2a2ddfa950b 100644 --- a/datafusion/src/datasource/object_store/local.rs +++ b/datafusion/src/datasource/object_store/local.rs @@ -27,6 +27,7 @@ use futures::{stream, AsyncRead, StreamExt}; use crate::datasource::object_store::{ FileMeta, FileMetaStream, ListEntryStream, ObjectReader, ObjectStore, }; +use crate::datasource::PartitionedFile; use crate::error::DataFusionError; use crate::error::Result; @@ -161,19 +162,22 @@ pub fn local_object_reader_stream(files: Vec) -> ObjectReaderStream { /// Helper method to convert a file location to a `LocalFileReader` pub fn local_object_reader(file: String) -> Arc { LocalFileSystem - .file_reader(local_file_meta(file).sized_file) + .file_reader(local_unpartitioned_file(file).file_meta.sized_file) .expect("File not found") } /// Helper method to fetch the file size and date at given path and create a `FileMeta` -pub fn local_file_meta(file: String) -> FileMeta { +pub fn local_unpartitioned_file(file: String) -> PartitionedFile { let metadata = fs::metadata(&file).expect("Local file metadata"); - FileMeta { - sized_file: SizedFile { - size: metadata.len(), - path: file, + PartitionedFile { + file_meta: FileMeta { + sized_file: SizedFile { + size: metadata.len(), + path: file, + }, + last_modified: metadata.modified().map(chrono::DateTime::from).ok(), }, - last_modified: metadata.modified().map(chrono::DateTime::from).ok(), + partition_values: vec![], } } diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index 2420040c08af..9bae1ebd59b8 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -203,7 +203,7 @@ impl ExecutionPlan for AvroExec { mod tests { use crate::datasource::object_store::local::{ - local_file_meta, local_object_reader_stream, LocalFileSystem, + local_object_reader_stream, local_unpartitioned_file, LocalFileSystem, }; use super::*; @@ -218,9 +218,7 @@ mod tests { let filename = format!("{}/avro/alltypes_plain.avro", testdata); let avro_exec = AvroExec::new( Arc::new(LocalFileSystem {}), - vec![vec![PartitionedFile { - file_meta: local_file_meta(filename.clone()), - }]], + vec![vec![local_unpartitioned_file(filename.clone())]], Statistics::default(), AvroFormat {} .infer_schema(local_object_reader_stream(vec![filename])) diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index fc82c8fd272e..3347198de4ba 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -209,7 +209,7 @@ impl ExecutionPlan for CsvExec { mod tests { use super::*; use crate::{ - datasource::object_store::local::{local_file_meta, LocalFileSystem}, + datasource::object_store::local::{local_unpartitioned_file, LocalFileSystem}, test::aggr_test_schema, }; use futures::StreamExt; @@ -222,9 +222,7 @@ mod tests { let path = format!("{}/csv/{}", testdata, filename); let csv = CsvExec::new( Arc::new(LocalFileSystem {}), - vec![vec![PartitionedFile { - file_meta: local_file_meta(path), - }]], + vec![vec![local_unpartitioned_file(path)]], Statistics::default(), schema, true, @@ -255,9 +253,7 @@ mod tests { let path = format!("{}/csv/{}", testdata, filename); let csv = CsvExec::new( Arc::new(LocalFileSystem {}), - vec![vec![PartitionedFile { - file_meta: local_file_meta(path), - }]], + vec![vec![local_unpartitioned_file(path)]], Statistics::default(), schema, true, diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index f9dde67fea2d..c1ee0cb87c56 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -169,7 +169,7 @@ mod tests { use crate::datasource::{ file_format::{json::JsonFormat, FileFormat}, object_store::local::{ - local_file_meta, local_object_reader_stream, LocalFileSystem, + local_object_reader_stream, local_unpartitioned_file, LocalFileSystem, }, }; @@ -189,9 +189,7 @@ mod tests { let path = format!("{}/1.json", TEST_DATA_BASE); let exec = NdJsonExec::new( Arc::new(LocalFileSystem {}), - vec![vec![PartitionedFile { - file_meta: local_file_meta(path.clone()), - }]], + vec![vec![local_unpartitioned_file(path.clone())]], Default::default(), infer_schema(path).await?, None, @@ -244,9 +242,7 @@ mod tests { let path = format!("{}/1.json", TEST_DATA_BASE); let exec = NdJsonExec::new( Arc::new(LocalFileSystem {}), - vec![vec![PartitionedFile { - file_meta: local_file_meta(path.clone()), - }]], + vec![vec![local_unpartitioned_file(path.clone())]], Default::default(), infer_schema(path).await?, Some(vec![0, 2]), diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index d07d2a945e8e..b86ed1dc43f8 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -519,7 +519,7 @@ mod tests { use crate::datasource::{ file_format::{parquet::ParquetFormat, FileFormat}, object_store::local::{ - local_file_meta, local_object_reader_stream, LocalFileSystem, + local_object_reader_stream, local_unpartitioned_file, LocalFileSystem, }, }; @@ -538,9 +538,7 @@ mod tests { let filename = format!("{}/alltypes_plain.parquet", testdata); let parquet_exec = ParquetExec::new( Arc::new(LocalFileSystem {}), - vec![vec![PartitionedFile { - file_meta: local_file_meta(filename.clone()), - }]], + vec![vec![local_unpartitioned_file(filename.clone())]], Statistics::default(), ParquetFormat::default() .infer_schema(local_object_reader_stream(vec![filename])) diff --git a/datafusion/src/test/mod.rs b/datafusion/src/test/mod.rs index f673eb065aaf..c13df55c05df 100644 --- a/datafusion/src/test/mod.rs +++ b/datafusion/src/test/mod.rs @@ -17,7 +17,7 @@ //! Common unit test utility methods -use crate::datasource::object_store::local::local_file_meta; +use crate::datasource::object_store::local::local_unpartitioned_file; use crate::datasource::{MemTable, PartitionedFile, TableProvider}; use crate::error::Result; use crate::logical_plan::{LogicalPlan, LogicalPlanBuilder}; @@ -98,11 +98,7 @@ pub fn create_partitioned_csv( let groups = files .into_iter() - .map(|f| { - vec![PartitionedFile { - file_meta: local_file_meta(f.to_str().unwrap().to_owned()), - }] - }) + .map(|f| vec![local_unpartitioned_file(f.to_str().unwrap().to_owned())]) .collect::>(); Ok((tmp_dir.into_path().to_str().unwrap().to_string(), groups)) From 2e829021b811e3cd433e1dfb31e8f3ac5c842ac1 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 02/17] [wip] implementing pruned_partition_list --- datafusion/src/datasource/listing.rs | 163 +++++++++++++++++++++++++-- 1 file changed, 155 insertions(+), 8 deletions(-) diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing.rs index cd00f2c7069b..6126e22c6b88 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing.rs @@ -20,22 +20,29 @@ use std::{any::Any, sync::Arc}; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::{ + datatypes::{DataType, Field, Schema, SchemaRef}, + record_batch::RecordBatch, +}; use async_trait::async_trait; -use futures::StreamExt; +use futures::{ + stream::{self, TryChunksError}, + StreamExt, TryStreamExt, +}; use crate::{ - datasource::PartitionedFile, + datasource::{MemTable, PartitionedFile}, error::{DataFusionError, Result}, - logical_plan::Expr, - physical_plan::{ExecutionPlan, Statistics}, + execution::context::ExecutionContext, + logical_plan::{self, Expr}, + physical_plan::{functions::Volatility, ExecutionPlan, Statistics}, }; use super::{ datasource::TableProviderFilterPushDown, file_format::{FileFormat, PhysicalPlanConfig}, get_statistics_with_limit, - object_store::ObjectStore, + object_store::{FileMeta, ObjectStore}, PartitionedFileStream, TableProvider, }; @@ -242,10 +249,13 @@ impl ListingTable { /// Discover the partitions on the given path and prune out files /// relative to irrelevant partitions using `filters` expressions +/// TODO for tables with many files (10k+), it will usually more efficient +/// to first list the folders relative to the first partition dimension, +/// prune those, then list only the contain of the remaining folders. async fn pruned_partition_list( store: &dyn ObjectStore, path: &str, - _filters: &[Expr], + filters: &[Expr], file_extension: &str, partition_names: &[String], ) -> Result { @@ -262,10 +272,63 @@ async fn pruned_partition_list( }), )) } else { - todo!("use filters to prune partitions") + let applicable_filters = filters + .iter() + .filter(|f| expr_applicable_for_cols(partition_names, f)); + + let schema = Arc::new(Schema::new( + partition_names + .iter() + .map(|pn| Field::new(pn, DataType::Utf8, false)) + .collect(), + )); + + let stream_schema = Arc::clone(&schema); + let stream_path = path.to_owned(); + // TODO avoid collecting but have a streaming memory table instead + let batches: Vec = store + .list_file_with_suffix(path, file_extension) + .await? + .try_chunks(64) + .map_err(|TryChunksError(_, e)| e) + .map_ok(move |metas| { + paths_to_batch(Arc::clone(&stream_schema), &stream_path, metas) + }) + .try_collect() + .await?; + + let mem_table = MemTable::try_new(schema, vec![batches])?; + + // Filter the partitions using a local datafusion context + // TODO having the external context would allow us to resolve `Volatility::Stable` + // scalar functions (`ScalarFunction` & `ScalarUDF`) and `ScalarVariable`s + let mut ctx = ExecutionContext::new(); + let mut df = ctx.read_table(Arc::new(mem_table))?; + for filter in applicable_filters { + df = df.filter(filter.clone())?; + } + let filtered_batches = df.collect().await?; + + Ok(Box::pin(stream::iter( + batches_to_paths(path, &filtered_batches) + .into_iter() + .map(Ok), + ))) } } +/// convert the paths of the files to a record batch with the following columns: +/// - one column by partition +/// - on column for the file size named `_df_part_file_size_` +fn paths_to_batch(schema: SchemaRef, path: &str, metas: Vec) -> RecordBatch { + RecordBatch::new_empty(schema) +} + +/// convert a set of record batches created by `paths_to_batch()` back to partitioned files. +fn batches_to_paths(path: &str, batches: &[RecordBatch]) -> Vec { + vec![] +} + fn split_files( partitioned_files: Vec, n: usize, @@ -280,6 +343,90 @@ fn split_files( .collect() } +fn expr_applicable_for_cols<'a>(col_names: &[String], expr: &'a Expr) -> bool { + match expr { + // leaf + Expr::Literal(_) => true, + // TODO how to handle qualified / unqualified names? + Expr::Column(logical_plan::Column { ref name, .. }) => col_names.contains(name), + // unary + Expr::Alias(child, _) + | Expr::Not(child) + | Expr::IsNotNull(child) + | Expr::IsNull(child) + | Expr::Negative(child) + | Expr::Cast { expr: child, .. } + | Expr::TryCast { expr: child, .. } => expr_applicable_for_cols(col_names, child), + // binary + Expr::BinaryExpr { + ref left, + ref right, + .. + } => { + expr_applicable_for_cols(col_names, left) + && expr_applicable_for_cols(col_names, right) + } + // ternary + Expr::Between { + expr: item, + low, + high, + .. + } => { + expr_applicable_for_cols(col_names, item) + && expr_applicable_for_cols(col_names, low) + && expr_applicable_for_cols(col_names, high) + } + // variadic + Expr::ScalarFunction { fun, args } => match fun.volatility() { + Volatility::Immutable => args + .iter() + .all(|arg| expr_applicable_for_cols(col_names, arg)), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable => false, + Volatility::Volatile => false, + }, + Expr::ScalarUDF { fun, args } => match fun.signature.volatility { + Volatility::Immutable => args + .iter() + .all(|arg| expr_applicable_for_cols(col_names, arg)), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable => false, + Volatility::Volatile => false, + }, + Expr::InList { + expr: item, list, .. + } => { + expr_applicable_for_cols(col_names, item) + && list.iter().all(|e| expr_applicable_for_cols(col_names, e)) + } + Expr::Case { + expr, + when_then_expr, + else_expr, + } => { + let expr_constant = expr + .as_ref() + .map(|e| expr_applicable_for_cols(col_names, e)) + .unwrap_or(true); + let else_constant = else_expr + .as_ref() + .map(|e| expr_applicable_for_cols(col_names, e)) + .unwrap_or(true); + let when_then_constant = when_then_expr.iter().all(|(w, th)| { + expr_applicable_for_cols(col_names, w) + && expr_applicable_for_cols(col_names, th) + }); + expr_constant && else_constant && when_then_constant + } + // TODO other expressions are not handled yet: + // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases + // - Can `Wildcard` be considered as a `Literal`? + // - ScalarVariable could be `applicable`, but that would require access to the context + _ => false, + } +} + #[cfg(test)] mod tests { use crate::{ From cbbf34752d614fec2d92cb0a3a1b63ef8a84b832 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 03/17] [feat] implement pruned_partition_list --- .../core/src/serde/logical_plan/to_proto.rs | 2 +- .../src/serde/physical_plan/from_proto.rs | 2 +- datafusion/src/datasource/listing/helpers.rs | 680 ++++++++++++++++++ datafusion/src/datasource/listing/mod.rs | 24 + .../{listing.rs => listing/table.rs} | 338 ++------- datafusion/src/datasource/object_store/mod.rs | 4 +- datafusion/src/test/object_store.rs | 2 +- 7 files changed, 781 insertions(+), 271 deletions(-) create mode 100644 datafusion/src/datasource/listing/helpers.rs create mode 100644 datafusion/src/datasource/listing/mod.rs rename datafusion/src/datasource/{listing.rs => listing/table.rs} (54%) diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index ae25d72d57f9..d2988197b225 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -756,7 +756,7 @@ impl TryInto for &LogicalPlan { .file_extension .clone(), partitions: listing_table.options().partitions.clone(), - path: listing_table.path().to_owned(), + path: listing_table.table_path().to_owned(), schema: Some(schema), projection, filters, diff --git a/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/ballista/rust/core/src/serde/physical_plan/from_proto.rs index 5864b6155282..85cb1d9bd36b 100644 --- a/ballista/rust/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -30,7 +30,7 @@ use crate::serde::protobuf::ShuffleReaderPartition; use crate::serde::scheduler::PartitionLocation; use crate::serde::{from_proto_binary_op, proto_error, protobuf, str_to_byte}; use crate::{convert_box_required, convert_required, into_required}; -use chrono::{DateTime, NaiveDateTime, TimeZone, Utc}; +use chrono::{TimeZone, Utc}; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::catalog::catalog::{ CatalogList, CatalogProvider, MemoryCatalogList, MemoryCatalogProvider, diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs new file mode 100644 index 000000000000..34ddb66bea75 --- /dev/null +++ b/datafusion/src/datasource/listing/helpers.rs @@ -0,0 +1,680 @@ +// 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. + +//! Helper functions for the table implementation + +use std::sync::Arc; + +use arrow::{ + array::{ + Array, ArrayBuilder, ArrayRef, Date64Array, Date64Builder, StringArray, + StringBuilder, UInt64Array, UInt64Builder, + }, + datatypes::{DataType, Field, Schema}, + record_batch::RecordBatch, +}; +use chrono::{TimeZone, Utc}; +use futures::{ + stream::{self, TryChunksError}, + StreamExt, TryStreamExt, +}; +use log::debug; + +use crate::{ + error::Result, + execution::context::ExecutionContext, + logical_plan::{self, Expr}, + physical_plan::functions::Volatility, + scalar::ScalarValue, +}; + +use crate::datasource::{ + object_store::{FileMeta, ObjectStore, SizedFile}, + MemTable, PartitionedFile, PartitionedFileStream, +}; + +const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_"; +const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_"; +const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_"; + +/// Partition the list of files into `n` groups +pub fn split_files( + partitioned_files: Vec, + n: usize, +) -> Vec> { + if partitioned_files.is_empty() { + return vec![]; + } + let mut chunk_size = partitioned_files.len() / n; + if partitioned_files.len() % n > 0 { + chunk_size += 1; + } + partitioned_files + .chunks(chunk_size) + .map(|c| c.to_vec()) + .collect() +} + +/// Discover the partitions on the given path and prune out files +/// relative to irrelevant partitions using `filters` expressions +/// TODO for tables with many files (10k+), it will usually more efficient +/// to first list the folders relative to the first partition dimension, +/// prune those, then list only the contain of the remaining folders. +pub async fn pruned_partition_list( + store: &dyn ObjectStore, + table_path: &str, + filters: &[Expr], + file_extension: &str, + partition_names: &[String], +) -> Result { + if partition_names.is_empty() { + Ok(Box::pin( + store + .list_file_with_suffix(table_path, file_extension) + .await? + .map(|f| { + Ok(PartitionedFile { + file_meta: f?, + partition_values: vec![], + }) + }), + )) + } else { + let applicable_filters = filters + .iter() + .filter(|f| expr_applicable_for_cols(partition_names, f)); + + let partition_names = partition_names.to_vec(); + let stream_path = table_path.to_owned(); + // TODO avoid collecting but have a streaming memory table instead + let batches: Vec = store + .list_file_with_suffix(table_path, file_extension) + .await? + .try_chunks(64) + .map_err(|TryChunksError(_, e)| e) + .map(move |metas| paths_to_batch(&partition_names, &stream_path, &metas?)) + .try_collect() + .await?; + + let mem_table = MemTable::try_new(batches[0].schema(), vec![batches])?; + + // Filter the partitions using a local datafusion context + // TODO having the external context would allow us to resolve `Volatility::Stable` + // scalar functions (`ScalarFunction` & `ScalarUDF`) and `ScalarVariable`s + let mut ctx = ExecutionContext::new(); + let mut df = ctx.read_table(Arc::new(mem_table))?; + for filter in applicable_filters { + df = df.filter(filter.clone())?; + } + let filtered_batches = df.collect().await?; + + Ok(Box::pin(stream::iter( + batches_to_paths(&filtered_batches).into_iter().map(Ok), + ))) + } +} + +/// convert the paths of the files to a record batch with the following columns: +/// - one column for the file size named `_df_part_file_size_` +/// - one column for with the original path named `_df_part_file_path_` +/// - one column for with the last modified date named `_df_part_file_modified_` +/// - ... one column by partition ... +/// Note: For the last modified date, this looses precisions higher than millisecond. +fn paths_to_batch( + partition_names: &[String], + table_path: &str, + metas: &[FileMeta], +) -> Result { + let mut key_builder = StringBuilder::new(metas.len()); + let mut length_builder = UInt64Builder::new(metas.len()); + let mut modified_builder = Date64Builder::new(metas.len()); + let mut partition_builders = partition_names + .iter() + .map(|_| StringBuilder::new(metas.len())) + .collect::>(); + for file_meta in metas { + if let Some(partition_values) = + parse_partitions_for_path(table_path, file_meta.path(), partition_names) + { + key_builder.append_value(file_meta.path())?; + length_builder.append_value(file_meta.size())?; + match file_meta.last_modified { + Some(lm) => modified_builder.append_value(lm.timestamp_millis())?, + None => modified_builder.append_null()?, + } + for (i, part_val) in partition_values.iter().enumerate() { + partition_builders[i].append_value(part_val)?; + } + } else { + debug!("No partitioning for path {}", file_meta.path()); + } + } + + // finish all builders + let mut col_arrays: Vec = vec![ + ArrayBuilder::finish(&mut key_builder), + ArrayBuilder::finish(&mut length_builder), + ArrayBuilder::finish(&mut modified_builder), + ]; + for mut partition_builder in partition_builders { + col_arrays.push(ArrayBuilder::finish(&mut partition_builder)); + } + + // put the schema together + let mut fields = vec![ + Field::new(FILE_SIZE_COLUMN_NAME, DataType::Utf8, false), + Field::new(FILE_PATH_COLUMN_NAME, DataType::UInt64, false), + Field::new(FILE_MODIFIED_COLUMN_NAME, DataType::Date64, false), + ]; + for pn in partition_names { + fields.push(Field::new(pn, DataType::Utf8, false)); + } + + let batch = RecordBatch::try_new(Arc::new(Schema::new(fields)), col_arrays)?; + Ok(batch) +} + +/// convert a set of record batches created by `paths_to_batch()` back to partitioned files. +fn batches_to_paths(batches: &[RecordBatch]) -> Vec { + batches + .iter() + .flat_map(|batch| { + let key_array = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let length_array = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + let modified_array = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + + (0..batch.num_rows()).map(move |row| PartitionedFile { + file_meta: FileMeta { + last_modified: match modified_array.is_null(row) { + false => Some(Utc.timestamp_millis(modified_array.value(row))), + true => None, + }, + sized_file: SizedFile { + path: key_array.value(row).to_owned(), + size: length_array.value(row), + }, + }, + partition_values: (3..batch.columns().len()) + .map(|col| { + ScalarValue::try_from_array(batch.column(col), row).unwrap() + }) + .collect(), + }) + }) + .collect() +} + +/// Extract the partition values for the given `file_path` (in the given `table_path`) +/// associated to the partitions defined by `partition_names` +fn parse_partitions_for_path<'a>( + table_path: &str, + file_path: &'a str, + partition_names: &[String], +) -> Option> { + let subpath = file_path.strip_prefix(table_path)?; + + // ignore whether table_path ended with "/" or not + let subpath = match subpath.strip_prefix('/') { + Some(subpath) => subpath, + None => subpath, + }; + + let mut part_values = vec![]; + for (path, pn) in subpath.split('/').zip(partition_names) { + if let Some(val) = path.strip_prefix(&format!("{}=", pn)) { + part_values.push(val); + } else { + return None; + } + } + Some(part_values) +} + +/// Check whether the given expression can be resolved using only the columns `col_names` +fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { + match expr { + // leaf + Expr::Literal(_) => true, + // TODO how to handle qualified / unqualified names? + Expr::Column(logical_plan::Column { ref name, .. }) => col_names.contains(name), + // unary + Expr::Alias(child, _) + | Expr::Not(child) + | Expr::IsNotNull(child) + | Expr::IsNull(child) + | Expr::Negative(child) + | Expr::Cast { expr: child, .. } + | Expr::TryCast { expr: child, .. } => expr_applicable_for_cols(col_names, child), + // binary + Expr::BinaryExpr { + ref left, + ref right, + .. + } => { + expr_applicable_for_cols(col_names, left) + && expr_applicable_for_cols(col_names, right) + } + // ternary + Expr::Between { + expr: item, + low, + high, + .. + } => { + expr_applicable_for_cols(col_names, item) + && expr_applicable_for_cols(col_names, low) + && expr_applicable_for_cols(col_names, high) + } + // variadic + Expr::ScalarFunction { fun, args } => match fun.volatility() { + Volatility::Immutable => args + .iter() + .all(|arg| expr_applicable_for_cols(col_names, arg)), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable => false, + Volatility::Volatile => false, + }, + Expr::ScalarUDF { fun, args } => match fun.signature.volatility { + Volatility::Immutable => args + .iter() + .all(|arg| expr_applicable_for_cols(col_names, arg)), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable => false, + Volatility::Volatile => false, + }, + Expr::InList { + expr: item, list, .. + } => { + expr_applicable_for_cols(col_names, item) + && list.iter().all(|e| expr_applicable_for_cols(col_names, e)) + } + Expr::Case { + expr, + when_then_expr, + else_expr, + } => { + let expr_constant = expr + .as_ref() + .map(|e| expr_applicable_for_cols(col_names, e)) + .unwrap_or(true); + let else_constant = else_expr + .as_ref() + .map(|e| expr_applicable_for_cols(col_names, e)) + .unwrap_or(true); + let when_then_constant = when_then_expr.iter().all(|(w, th)| { + expr_applicable_for_cols(col_names, w) + && expr_applicable_for_cols(col_names, th) + }); + expr_constant && else_constant && when_then_constant + } + // TODO other expressions are not handled yet: + // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases + // - Can `Wildcard` be considered as a `Literal`? + // - ScalarVariable could be `applicable`, but that would require access to the context + _ => false, + } +} + +#[cfg(test)] +mod tests { + use crate::{ + logical_plan::{case, col, lit}, + test::object_store::TestObjectStore, + }; + + use super::*; + + #[test] + fn test_split_files() { + let new_partitioned_file = |path: &str| PartitionedFile::new(path.to_owned(), 10); + let files = vec![ + new_partitioned_file("a"), + new_partitioned_file("b"), + new_partitioned_file("c"), + new_partitioned_file("d"), + new_partitioned_file("e"), + ]; + + let chunks = split_files(files.clone(), 1); + assert_eq!(1, chunks.len()); + assert_eq!(5, chunks[0].len()); + + let chunks = split_files(files.clone(), 2); + assert_eq!(2, chunks.len()); + assert_eq!(3, chunks[0].len()); + assert_eq!(2, chunks[1].len()); + + let chunks = split_files(files.clone(), 5); + assert_eq!(5, chunks.len()); + assert_eq!(1, chunks[0].len()); + assert_eq!(1, chunks[1].len()); + assert_eq!(1, chunks[2].len()); + assert_eq!(1, chunks[3].len()); + assert_eq!(1, chunks[4].len()); + + let chunks = split_files(files, 123); + assert_eq!(5, chunks.len()); + assert_eq!(1, chunks[0].len()); + assert_eq!(1, chunks[1].len()); + assert_eq!(1, chunks[2].len()); + assert_eq!(1, chunks[3].len()); + assert_eq!(1, chunks[4].len()); + + let chunks = split_files(vec![], 2); + assert_eq!(0, chunks.len()); + } + + #[tokio::test] + async fn test_pruned_partition_list_empty() { + let store = TestObjectStore::new_arc(&[ + ("tablepath/mypartition=val1/notparquetfile", 100), + ("tablepath/file.parquet", 100), + ]); + let filter = Expr::eq(col("mypartition"), lit("val1")); + let pruned = pruned_partition_list( + store.as_ref(), + "tablepath/", + &[filter], + ".parquet", + &[String::from("mypartition")], + ) + .await + .expect("partition pruning failed") + .collect::>() + .await; + + assert_eq!(pruned.len(), 0); + } + + #[tokio::test] + async fn test_pruned_partition_list() { + let store = TestObjectStore::new_arc(&[ + ("tablepath/mypartition=val1/file.parquet", 100), + ("tablepath/mypartition=val2/file.parquet", 100), + ("tablepath/mypartition=val1/other=val3/file.parquet", 100), + ]); + let filter = Expr::eq(col("mypartition"), lit("val1")); + let pruned = pruned_partition_list( + store.as_ref(), + "tablepath/", + &[filter], + ".parquet", + &[String::from("mypartition")], + ) + .await + .expect("partition pruning failed") + .collect::>() + .await; + + assert_eq!(pruned.len(), 2); + let f1 = pruned[0].as_ref().expect("first item not an error"); + assert_eq!( + &f1.file_meta.sized_file.path, + "tablepath/mypartition=val1/file.parquet" + ); + assert_eq!( + &f1.partition_values, + &[ScalarValue::Utf8(Some(String::from("val1"))),] + ); + let f2 = pruned[1].as_ref().expect("second item not an error"); + assert_eq!( + &f2.file_meta.sized_file.path, + "tablepath/mypartition=val1/other=val3/file.parquet" + ); + assert_eq!( + &f2.partition_values, + &[ScalarValue::Utf8(Some(String::from("val1"))),] + ); + } + + #[tokio::test] + async fn test_pruned_partition_list_multi() { + let store = TestObjectStore::new_arc(&[ + ("tablepath/part1=p1v1/file.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v1/file1.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v1/file2.parquet", 100), + ("tablepath/part1=p1v3/part2=p2v2/file2.parquet", 100), + ]); + let filter1 = Expr::eq(col("part1"), lit("p1v2")); + // filter2 cannot be resolved at partition pruning + let filter2 = Expr::eq(col("part2"), col("other")); + let pruned = pruned_partition_list( + store.as_ref(), + "tablepath/", + &[filter1, filter2], + ".parquet", + &[String::from("part1"), String::from("part2")], + ) + .await + .expect("partition pruning failed") + .collect::>() + .await; + + assert_eq!(pruned.len(), 2); + let f1 = pruned[0].as_ref().expect("first item not an error"); + assert_eq!( + &f1.file_meta.sized_file.path, + "tablepath/part1=p1v2/part2=p2v1/file1.parquet" + ); + assert_eq!( + &f1.partition_values, + &[ + ScalarValue::Utf8(Some(String::from("p1v2"))), + ScalarValue::Utf8(Some(String::from("p2v1"))) + ] + ); + let f2 = pruned[1].as_ref().expect("second item not an error"); + assert_eq!( + &f2.file_meta.sized_file.path, + "tablepath/part1=p1v2/part2=p2v1/file2.parquet" + ); + assert_eq!( + &f2.partition_values, + &[ + ScalarValue::Utf8(Some(String::from("p1v2"))), + ScalarValue::Utf8(Some(String::from("p2v1"))) + ] + ); + } + + #[test] + fn test_parse_partitions_for_path() { + assert_eq!( + Some(vec![]), + parse_partitions_for_path("bucket/mytable", "bucket/mytable/file.csv", &[]) + ); + assert_eq!( + None, + parse_partitions_for_path( + "bucket/othertable", + "bucket/mytable/file.csv", + &[] + ) + ); + assert_eq!( + None, + parse_partitions_for_path( + "bucket/mytable", + "bucket/mytable/file.csv", + &[String::from("mypartition")] + ) + ); + assert_eq!( + Some(vec!["v1"]), + parse_partitions_for_path( + "bucket/mytable", + "bucket/mytable/mypartition=v1/file.csv", + &[String::from("mypartition")] + ) + ); + assert_eq!( + Some(vec!["v1"]), + parse_partitions_for_path( + "bucket/mytable/", + "bucket/mytable/mypartition=v1/file.csv", + &[String::from("mypartition")] + ) + ); + // Only hive style partitioning supported for now: + assert_eq!( + None, + parse_partitions_for_path( + "bucket/mytable", + "bucket/mytable/v1/file.csv", + &[String::from("mypartition")] + ) + ); + assert_eq!( + Some(vec!["v1", "v2"]), + parse_partitions_for_path( + "bucket/mytable", + "bucket/mytable/mypartition=v1/otherpartition=v2/file.csv", + &[String::from("mypartition"), String::from("otherpartition")] + ) + ); + assert_eq!( + Some(vec!["v1"]), + parse_partitions_for_path( + "bucket/mytable", + "bucket/mytable/mypartition=v1/otherpartition=v2/file.csv", + &[String::from("mypartition")] + ) + ); + } + + #[test] + fn test_path_batch_roundtrip_no_partiton() { + let files = vec![ + FileMeta { + sized_file: SizedFile { + path: String::from("mybucket/tablepath/part1=val1/file.parquet"), + size: 100, + }, + last_modified: Some(Utc.timestamp_millis(1634722979123)), + }, + FileMeta { + sized_file: SizedFile { + path: String::from("mybucket/tablepath/part1=val2/file.parquet"), + size: 100, + }, + last_modified: None, + }, + ]; + + let batches = paths_to_batch(&[], "mybucket/tablepath", &files) + .expect("Serialization of file list to batch failed"); + + let parsed_files = batches_to_paths(&[batches]); + assert_eq!(parsed_files.len(), 2); + assert_eq!(&parsed_files[0].partition_values, &[]); + assert_eq!(&parsed_files[1].partition_values, &[]); + + let parsed_metas = parsed_files + .into_iter() + .map(|pf| pf.file_meta) + .collect::>(); + assert_eq!(parsed_metas, files); + } + + #[test] + fn test_path_batch_roundtrip_with_partition() { + let files = vec![ + FileMeta { + sized_file: SizedFile { + path: String::from("mybucket/tablepath/part1=val1/file.parquet"), + size: 100, + }, + last_modified: Some(Utc.timestamp_millis(1634722979123)), + }, + FileMeta { + sized_file: SizedFile { + path: String::from("mybucket/tablepath/part1=val2/file.parquet"), + size: 100, + }, + last_modified: None, + }, + ]; + + let batches = + paths_to_batch(&[String::from("part1")], "mybucket/tablepath", &files) + .expect("Serialization of file list to batch failed"); + + let parsed_files = batches_to_paths(&[batches]); + assert_eq!(parsed_files.len(), 2); + assert_eq!( + &parsed_files[0].partition_values, + &[ScalarValue::Utf8(Some(String::from("val1")))] + ); + assert_eq!( + &parsed_files[1].partition_values, + &[ScalarValue::Utf8(Some(String::from("val2")))] + ); + + let parsed_metas = parsed_files + .into_iter() + .map(|pf| pf.file_meta) + .collect::>(); + assert_eq!(parsed_metas, files); + } + + #[test] + fn test_expr_applicable_for_cols() { + assert!(expr_applicable_for_cols( + &[String::from("c1")], + &Expr::eq(col("c1"), lit("value")) + )); + assert!(!expr_applicable_for_cols( + &[String::from("c1")], + &Expr::eq(col("c2"), lit("value")) + )); + assert!(!expr_applicable_for_cols( + &[String::from("c1")], + &Expr::eq(col("c1"), col("c2")) + )); + assert!(expr_applicable_for_cols( + &[String::from("c1"), String::from("c2")], + &Expr::eq(col("c1"), col("c2")) + )); + assert!(expr_applicable_for_cols( + &[String::from("c1"), String::from("c2")], + &(Expr::eq(col("c1"), col("c2").alias("c2_alias"))).not() + )); + assert!(expr_applicable_for_cols( + &[String::from("c1"), String::from("c2")], + &(case(col("c1")) + .when(lit("v1"), lit(true)) + .otherwise(lit(false)) + .expect("valid case expr")) + )); + // static expression not relvant in this context but we + // test it as an edge case anyway in case we want to generalize + // this helper function + assert!(expr_applicable_for_cols(&[], &lit(true))); + } +} diff --git a/datafusion/src/datasource/listing/mod.rs b/datafusion/src/datasource/listing/mod.rs new file mode 100644 index 000000000000..c8b92418ba2f --- /dev/null +++ b/datafusion/src/datasource/listing/mod.rs @@ -0,0 +1,24 @@ +// 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. + +//! A table that uses the `ObjectStore` listing capability +//! to get the list of files to process. + +mod helpers; +mod table; + +pub use table::{ListingOptions, ListingTable}; diff --git a/datafusion/src/datasource/listing.rs b/datafusion/src/datasource/listing/table.rs similarity index 54% rename from datafusion/src/datasource/listing.rs rename to datafusion/src/datasource/listing/table.rs index 6126e22c6b88..e83e85a61863 100644 --- a/datafusion/src/datasource/listing.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -15,37 +15,30 @@ // specific language governing permissions and limitations // under the License. -//! A table that uses the `ObjectStore` listing capability -//! to get the list of files to process. +//! The table implementation. use std::{any::Any, sync::Arc}; -use arrow::{ - datatypes::{DataType, Field, Schema, SchemaRef}, - record_batch::RecordBatch, -}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use async_trait::async_trait; -use futures::{ - stream::{self, TryChunksError}, - StreamExt, TryStreamExt, -}; +use futures::StreamExt; use crate::{ - datasource::{MemTable, PartitionedFile}, - error::{DataFusionError, Result}, - execution::context::ExecutionContext, - logical_plan::{self, Expr}, - physical_plan::{functions::Volatility, ExecutionPlan, Statistics}, + error::Result, + logical_plan::Expr, + physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}, }; -use super::{ +use crate::datasource::{ datasource::TableProviderFilterPushDown, file_format::{FileFormat, PhysicalPlanConfig}, get_statistics_with_limit, - object_store::{FileMeta, ObjectStore}, - PartitionedFileStream, TableProvider, + object_store::ObjectStore, + PartitionedFile, TableProvider, }; +use super::helpers::{pruned_partition_list, split_files}; + /// Options for creating a `ListingTable` pub struct ListingOptions { /// A suffix on which files should be filtered (leave empty to @@ -116,7 +109,7 @@ impl ListingOptions { /// or file system listing capability to get the list of files. pub struct ListingTable { object_store: Arc, - path: String, + table_path: String, schema: SchemaRef, options: ListingOptions, } @@ -125,14 +118,14 @@ impl ListingTable { /// Create new table that lists the FS to get the files to scan. pub fn new( object_store: Arc, - path: String, + table_path: String, // the schema must be resolved before creating the table schema: SchemaRef, options: ListingOptions, ) -> Self { Self { object_store, - path, + table_path, schema, options, } @@ -143,8 +136,8 @@ impl ListingTable { &self.object_store } /// Get path ref - pub fn path(&self) -> &str { - &self.path + pub fn table_path(&self) -> &str { + &self.table_path } /// Get options ref pub fn options(&self) -> &ListingOptions { @@ -171,6 +164,19 @@ impl TableProvider for ListingTable { ) -> Result> { let (partitioned_file_lists, statistics) = self.list_files_for_scan(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 = match &projection { + None => schema, + Some(p) => Arc::new(Schema::new( + p.iter().map(|i| schema.field(*i).clone()).collect(), + )), + }; + return Ok(Arc::new(EmptyExec::new(false, projected_schema))); + } + // create the execution plan self.options .format @@ -206,7 +212,7 @@ impl ListingTable { // list files (with partitions) let file_list = pruned_partition_list( self.object_store.as_ref(), - &self.path, + &self.table_path, filters, &self.options.file_extension, &self.options.partitions, @@ -233,13 +239,6 @@ impl ListingTable { let (files, statistics) = get_statistics_with_limit(files, self.schema(), limit).await?; - if files.is_empty() { - return Err(DataFusionError::Plan(format!( - "No files found at {} with file extension {}", - self.path, self.options.file_extension, - ))); - } - Ok(( split_files(files, self.options.target_partitions), statistics, @@ -247,235 +246,19 @@ impl ListingTable { } } -/// Discover the partitions on the given path and prune out files -/// relative to irrelevant partitions using `filters` expressions -/// TODO for tables with many files (10k+), it will usually more efficient -/// to first list the folders relative to the first partition dimension, -/// prune those, then list only the contain of the remaining folders. -async fn pruned_partition_list( - store: &dyn ObjectStore, - path: &str, - filters: &[Expr], - file_extension: &str, - partition_names: &[String], -) -> Result { - if partition_names.is_empty() { - Ok(Box::pin( - store - .list_file_with_suffix(path, file_extension) - .await? - .map(|f| { - Ok(PartitionedFile { - file_meta: f?, - partition_values: vec![], - }) - }), - )) - } else { - let applicable_filters = filters - .iter() - .filter(|f| expr_applicable_for_cols(partition_names, f)); - - let schema = Arc::new(Schema::new( - partition_names - .iter() - .map(|pn| Field::new(pn, DataType::Utf8, false)) - .collect(), - )); - - let stream_schema = Arc::clone(&schema); - let stream_path = path.to_owned(); - // TODO avoid collecting but have a streaming memory table instead - let batches: Vec = store - .list_file_with_suffix(path, file_extension) - .await? - .try_chunks(64) - .map_err(|TryChunksError(_, e)| e) - .map_ok(move |metas| { - paths_to_batch(Arc::clone(&stream_schema), &stream_path, metas) - }) - .try_collect() - .await?; - - let mem_table = MemTable::try_new(schema, vec![batches])?; - - // Filter the partitions using a local datafusion context - // TODO having the external context would allow us to resolve `Volatility::Stable` - // scalar functions (`ScalarFunction` & `ScalarUDF`) and `ScalarVariable`s - let mut ctx = ExecutionContext::new(); - let mut df = ctx.read_table(Arc::new(mem_table))?; - for filter in applicable_filters { - df = df.filter(filter.clone())?; - } - let filtered_batches = df.collect().await?; - - Ok(Box::pin(stream::iter( - batches_to_paths(path, &filtered_batches) - .into_iter() - .map(Ok), - ))) - } -} - -/// convert the paths of the files to a record batch with the following columns: -/// - one column by partition -/// - on column for the file size named `_df_part_file_size_` -fn paths_to_batch(schema: SchemaRef, path: &str, metas: Vec) -> RecordBatch { - RecordBatch::new_empty(schema) -} - -/// convert a set of record batches created by `paths_to_batch()` back to partitioned files. -fn batches_to_paths(path: &str, batches: &[RecordBatch]) -> Vec { - vec![] -} - -fn split_files( - partitioned_files: Vec, - n: usize, -) -> Vec> { - let mut chunk_size = partitioned_files.len() / n; - if partitioned_files.len() % n > 0 { - chunk_size += 1; - } - partitioned_files - .chunks(chunk_size) - .map(|c| c.to_vec()) - .collect() -} - -fn expr_applicable_for_cols<'a>(col_names: &[String], expr: &'a Expr) -> bool { - match expr { - // leaf - Expr::Literal(_) => true, - // TODO how to handle qualified / unqualified names? - Expr::Column(logical_plan::Column { ref name, .. }) => col_names.contains(name), - // unary - Expr::Alias(child, _) - | Expr::Not(child) - | Expr::IsNotNull(child) - | Expr::IsNull(child) - | Expr::Negative(child) - | Expr::Cast { expr: child, .. } - | Expr::TryCast { expr: child, .. } => expr_applicable_for_cols(col_names, child), - // binary - Expr::BinaryExpr { - ref left, - ref right, - .. - } => { - expr_applicable_for_cols(col_names, left) - && expr_applicable_for_cols(col_names, right) - } - // ternary - Expr::Between { - expr: item, - low, - high, - .. - } => { - expr_applicable_for_cols(col_names, item) - && expr_applicable_for_cols(col_names, low) - && expr_applicable_for_cols(col_names, high) - } - // variadic - Expr::ScalarFunction { fun, args } => match fun.volatility() { - Volatility::Immutable => args - .iter() - .all(|arg| expr_applicable_for_cols(col_names, arg)), - // TODO: Stable functions could be `applicable`, but that would require access to the context - Volatility::Stable => false, - Volatility::Volatile => false, - }, - Expr::ScalarUDF { fun, args } => match fun.signature.volatility { - Volatility::Immutable => args - .iter() - .all(|arg| expr_applicable_for_cols(col_names, arg)), - // TODO: Stable functions could be `applicable`, but that would require access to the context - Volatility::Stable => false, - Volatility::Volatile => false, - }, - Expr::InList { - expr: item, list, .. - } => { - expr_applicable_for_cols(col_names, item) - && list.iter().all(|e| expr_applicable_for_cols(col_names, e)) - } - Expr::Case { - expr, - when_then_expr, - else_expr, - } => { - let expr_constant = expr - .as_ref() - .map(|e| expr_applicable_for_cols(col_names, e)) - .unwrap_or(true); - let else_constant = else_expr - .as_ref() - .map(|e| expr_applicable_for_cols(col_names, e)) - .unwrap_or(true); - let when_then_constant = when_then_expr.iter().all(|(w, th)| { - expr_applicable_for_cols(col_names, w) - && expr_applicable_for_cols(col_names, th) - }); - expr_constant && else_constant && when_then_constant - } - // TODO other expressions are not handled yet: - // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases - // - Can `Wildcard` be considered as a `Literal`? - // - ScalarVariable could be `applicable`, but that would require access to the context - _ => false, - } -} - #[cfg(test)] mod tests { use crate::{ datasource::{ file_format::{avro::AvroFormat, parquet::ParquetFormat}, - object_store::{local::LocalFileSystem, ObjectStore}, + object_store::local::LocalFileSystem, }, + logical_plan::{col, lit}, test::object_store::TestObjectStore, }; use super::*; - #[test] - fn test_split_files() { - let new_partitioned_file = |path: &str| PartitionedFile::new(path.to_owned(), 10); - let files = vec![ - new_partitioned_file("a"), - new_partitioned_file("b"), - new_partitioned_file("c"), - new_partitioned_file("d"), - new_partitioned_file("e"), - ]; - - let chunks = split_files(files.clone(), 1); - assert_eq!(1, chunks.len()); - assert_eq!(5, chunks[0].len()); - - let chunks = split_files(files.clone(), 2); - assert_eq!(2, chunks.len()); - assert_eq!(3, chunks[0].len()); - assert_eq!(2, chunks[1].len()); - - let chunks = split_files(files.clone(), 5); - assert_eq!(5, chunks.len()); - assert_eq!(1, chunks[0].len()); - assert_eq!(1, chunks[1].len()); - assert_eq!(1, chunks[2].len()); - assert_eq!(1, chunks[3].len()); - assert_eq!(1, chunks[4].len()); - - let chunks = split_files(files, 123); - assert_eq!(5, chunks.len()); - assert_eq!(1, chunks[0].len()); - assert_eq!(1, chunks[1].len()); - assert_eq!(1, chunks[2].len()); - assert_eq!(1, chunks[3].len()); - assert_eq!(1, chunks[4].len()); - } - #[tokio::test] async fn read_single_file() -> Result<()> { let table = load_table("alltypes_plain.parquet").await?; @@ -496,9 +279,38 @@ mod tests { } #[tokio::test] - async fn file_listings() -> Result<()> { + async fn read_empty_table() -> Result<()> { + let store = TestObjectStore::new_arc(&[("table/p1=v1/file.avro", 100)]); + + let opt = ListingOptions { + file_extension: ".avro".to_owned(), + format: Arc::new(AvroFormat {}), + partitions: vec![String::from("p1")], + target_partitions: 4, + collect_stat: true, + }; + + let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + + let table = ListingTable::new(store, "table/".to_owned(), Arc::new(schema), opt); + + // this will filter out the only file in the store + let filter = Expr::not_eq(col("p1"), lit("v1")); + + let scan = table + .scan(&None, 1024, &[filter], None) + .await + .expect("Empty execution plan"); + + assert!(scan.as_any().is::()); + + Ok(()) + } + + #[tokio::test] + async fn test_assert_list_files_for_scan_grouping() -> Result<()> { // more expected partitions than files - assert_partitioning( + assert_list_files_for_scan_grouping( &[ "bucket/key-prefix/file0", "bucket/key-prefix/file1", @@ -513,7 +325,7 @@ mod tests { .await?; // as many expected partitions as files - assert_partitioning( + assert_list_files_for_scan_grouping( &[ "bucket/key-prefix/file0", "bucket/key-prefix/file1", @@ -527,7 +339,7 @@ mod tests { .await?; // more files as expected partitions - assert_partitioning( + assert_list_files_for_scan_grouping( &[ "bucket/key-prefix/file0", "bucket/key-prefix/file1", @@ -541,13 +353,11 @@ mod tests { ) .await?; - // no files - assert_partitioning(&[], "bucket/key-prefix/", 2, 0) - .await - .expect_err("no files"); + // no files => no groups + assert_list_files_for_scan_grouping(&[], "bucket/key-prefix/", 2, 0).await?; // files that don't match the prefix - assert_partitioning( + assert_list_files_for_scan_grouping( &[ "bucket/key-prefix/file0", "bucket/key-prefix/file1", @@ -583,13 +393,13 @@ mod tests { /// Check that the files listed by the table match the specified `output_partitioning` /// when the object store contains `files`. - async fn assert_partitioning( + async fn assert_list_files_for_scan_grouping( files: &[&str], table_prefix: &str, target_partitions: usize, output_partitioning: usize, ) -> Result<()> { - let mock_store: Arc = + let mock_store = TestObjectStore::new_arc(&files.iter().map(|f| (*f, 10)).collect::>()); let format = AvroFormat {}; @@ -604,12 +414,8 @@ mod tests { let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table = ListingTable::new( - Arc::clone(&mock_store), - table_prefix.to_owned(), - Arc::new(schema), - opt, - ); + let table = + ListingTable::new(mock_store, table_prefix.to_owned(), Arc::new(schema), opt); let (file_list, _) = table.list_files_for_scan(&[], None).await?; diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs index 61bc47dc462c..59e184103d2a 100644 --- a/datafusion/src/datasource/object_store/mod.rs +++ b/datafusion/src/datasource/object_store/mod.rs @@ -70,7 +70,7 @@ pub enum ListEntry { } /// The path and size of the file. -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub struct SizedFile { /// Path of the file. It is relative to the current object /// store (it does not specify the `xx://` scheme). @@ -82,7 +82,7 @@ pub struct SizedFile { /// Description of a file as returned by the listing command of a /// given object store. The resulting path is relative to the /// object store that generated it. -#[derive(Debug, Clone)] +#[derive(Debug, Clone, PartialEq)] pub struct FileMeta { /// The path and size of the file. pub sized_file: SizedFile, diff --git a/datafusion/src/test/object_store.rs b/datafusion/src/test/object_store.rs index 4020b999f7d0..f82cfdbb8e99 100644 --- a/datafusion/src/test/object_store.rs +++ b/datafusion/src/test/object_store.rs @@ -41,7 +41,7 @@ pub struct TestObjectStore { } impl TestObjectStore { - pub fn new_arc(files: &[(&str, u64)]) -> Arc { + pub fn new_arc(files: &[(&str, u64)]) -> Arc { Arc::new(Self { files: files.iter().map(|f| (f.0.to_owned(), f.1)).collect(), }) From f74ee678f6b5b55d690e53029d5ab8dd15c440d9 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 04/17] [fix] outdated comment --- datafusion/src/datasource/listing/table.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index e83e85a61863..3c512ee955a5 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -52,7 +52,6 @@ pub struct ListingOptions { /// - If there is a third level of partitioning it will be ignored. /// - Files that don't follow this partitioning will be ignored. /// Note that only `DataType::Utf8` is supported for the column type. - /// TODO implement case where partitions.len() > 0 pub partitions: Vec, /// Set true to try to guess statistics from the files. /// This can add a lot of overhead as it will usually require files @@ -110,6 +109,7 @@ impl ListingOptions { pub struct ListingTable { object_store: Arc, table_path: String, + /// the schema also contains the partition columns schema: SchemaRef, options: ListingOptions, } From fb15b3b872a3579e46ea6ee89f69351939855b3a Mon Sep 17 00:00:00 2001 From: rdettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 05/17] [fix] adding extra partitioning col for csv avro json parquet exec still TODO --- ballista/rust/core/proto/ballista.proto | 5 +- .../core/src/serde/logical_plan/from_proto.rs | 2 +- .../core/src/serde/logical_plan/to_proto.rs | 5 +- .../src/serde/physical_plan/from_proto.rs | 67 +++++----- .../core/src/serde/physical_plan/to_proto.rs | 3 + benchmarks/src/bin/tpch.rs | 2 +- datafusion/src/datasource/file_format/avro.rs | 39 +++--- datafusion/src/datasource/file_format/csv.rs | 41 +++---- datafusion/src/datasource/file_format/json.rs | 39 +++--- datafusion/src/datasource/file_format/mod.rs | 13 +- .../src/datasource/file_format/parquet.rs | 42 +++---- datafusion/src/datasource/listing/helpers.rs | 32 ++--- datafusion/src/datasource/listing/table.rs | 37 +++--- datafusion/src/datasource/mod.rs | 1 + datafusion/src/execution/context.rs | 4 +- datafusion/src/execution/options.rs | 4 +- datafusion/src/logical_plan/builder.rs | 2 +- .../src/physical_optimizer/repartition.rs | 43 ++++--- .../src/physical_plan/coalesce_partitions.rs | 18 +-- .../src/physical_plan/file_format/avro.rs | 64 +++++----- .../src/physical_plan/file_format/csv.rs | 86 ++++++------- .../physical_plan/file_format/file_stream.rs | 116 +++++++++++++++--- .../src/physical_plan/file_format/json.rs | 82 ++++++------- .../src/physical_plan/file_format/mod.rs | 46 ++++++- .../src/physical_plan/file_format/parquet.rs | 103 ++++++---------- datafusion/src/physical_plan/filter.rs | 20 +-- datafusion/src/physical_plan/limit.rs | 18 +-- datafusion/src/physical_plan/projection.rs | 20 +-- datafusion/src/physical_plan/sort.rs | 20 +-- .../physical_plan/sort_preserving_merge.rs | 35 +++--- datafusion/src/physical_plan/union.rs | 35 +++--- datafusion/src/physical_plan/windows/mod.rs | 18 +-- 32 files changed, 608 insertions(+), 454 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index ce3bdb7cd95b..7ef00bdb3277 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -295,7 +295,7 @@ message ListingTableScanNode { ProjectionColumns projection = 4; Schema schema = 5; repeated LogicalExprNode filters = 6; - repeated string partitions = 7; + repeated string table_partition_dims = 7; bool collect_stat = 8; uint32 target_partitions = 9; oneof FileFormatType { @@ -621,6 +621,7 @@ message ParquetScanExecNode { repeated uint32 projection = 6; ScanLimit limit = 7; Statistics statistics = 8; + repeated string table_partition_dims = 9; } message CsvScanExecNode { @@ -632,6 +633,7 @@ message CsvScanExecNode { repeated uint32 projection = 6; ScanLimit limit = 7; Statistics statistics = 8; + repeated string table_partition_dims = 9; } message AvroScanExecNode { @@ -641,6 +643,7 @@ message AvroScanExecNode { repeated uint32 projection = 6; ScanLimit limit = 7; Statistics statistics = 8; + repeated string table_partition_dims = 9; } enum PartitionMode { diff --git a/ballista/rust/core/src/serde/logical_plan/from_proto.rs b/ballista/rust/core/src/serde/logical_plan/from_proto.rs index 26231c5e25c7..3b25d6fbeaab 100644 --- a/ballista/rust/core/src/serde/logical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/from_proto.rs @@ -191,7 +191,7 @@ impl TryInto for &protobuf::LogicalPlanNode { let options = ListingOptions { file_extension: scan.file_extension.clone(), format: file_format, - partitions: scan.partitions.clone(), + table_partition_dims: scan.table_partition_dims.clone(), collect_stat: scan.collect_stat, target_partitions: scan.target_partitions as usize, }; diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index d2988197b225..442fda46ee9a 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -755,7 +755,10 @@ impl TryInto for &LogicalPlan { .options() .file_extension .clone(), - partitions: listing_table.options().partitions.clone(), + table_partition_dims: listing_table + .options() + .table_partition_dims + .clone(), path: listing_table.table_path().to_owned(), schema: Some(schema), projection, diff --git a/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/ballista/rust/core/src/serde/physical_plan/from_proto.rs index 85cb1d9bd36b..861be3e29954 100644 --- a/ballista/rust/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -35,6 +35,7 @@ use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::catalog::catalog::{ CatalogList, CatalogProvider, MemoryCatalogList, MemoryCatalogProvider, }; +use datafusion::datasource::file_format::PhysicalPlanConfig; use datafusion::datasource::object_store::local::LocalFileSystem; use datafusion::datasource::object_store::{FileMeta, ObjectStoreRegistry, SizedFile}; use datafusion::datasource::PartitionedFile; @@ -124,18 +125,22 @@ impl TryInto> for &protobuf::PhysicalPlanNode { let statistics = convert_required!(scan.statistics)?; Ok(Arc::new(CsvExec::new( - Arc::new(LocalFileSystem {}), - scan.file_groups - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?, - statistics, - schema, + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: schema, + file_groups: scan + .file_groups + .iter() + .map(|f| f.try_into()) + .collect::, _>>()?, + statistics, + projection: Some(projection), + batch_size: scan.batch_size as usize, + limit: scan.limit.as_ref().map(|sl| sl.limit as usize), + table_partition_dims: vec![], + }, scan.has_header, str_to_byte(&scan.delimiter)?, - Some(projection), - scan.batch_size as usize, - scan.limit.as_ref().map(|sl| sl.limit as usize), ))) } PhysicalPlanType::ParquetScan(scan) => { @@ -144,18 +149,22 @@ impl TryInto> for &protobuf::PhysicalPlanNode { let statistics = convert_required!(scan.statistics)?; Ok(Arc::new(ParquetExec::new( - Arc::new(LocalFileSystem {}), - scan.file_groups - .iter() - .map(|p| p.try_into()) - .collect::, _>>()?, - statistics, - schema, - Some(projection), + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: schema, + file_groups: scan + .file_groups + .iter() + .map(|f| f.try_into()) + .collect::, _>>()?, + statistics, + projection: Some(projection), + batch_size: scan.batch_size as usize, + limit: scan.limit.as_ref().map(|sl| sl.limit as usize), + table_partition_dims: scan.table_partition_dims.clone(), + }, // TODO predicate should be de-serialized None, - scan.batch_size as usize, - scan.limit.as_ref().map(|sl| sl.limit as usize), ))) } PhysicalPlanType::AvroScan(scan) => { @@ -163,18 +172,20 @@ impl TryInto> for &protobuf::PhysicalPlanNode { let projection = scan.projection.iter().map(|i| *i as usize).collect(); let statistics = convert_required!(scan.statistics)?; - Ok(Arc::new(AvroExec::new( - Arc::new(LocalFileSystem {}), - scan.file_groups + Ok(Arc::new(AvroExec::new(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: schema, + file_groups: scan + .file_groups .iter() .map(|f| f.try_into()) .collect::, _>>()?, statistics, - schema, - Some(projection), - scan.batch_size as usize, - scan.limit.as_ref().map(|sl| sl.limit as usize), - ))) + projection: Some(projection), + batch_size: scan.batch_size as usize, + limit: scan.limit.as_ref().map(|sl| sl.limit as usize), + table_partition_dims: vec![], + }))) } PhysicalPlanType::CoalesceBatches(coalesce_batches) => { let input: Arc = diff --git a/ballista/rust/core/src/serde/physical_plan/to_proto.rs b/ballista/rust/core/src/serde/physical_plan/to_proto.rs index 3e65b6484cf6..3c061a029b7b 100644 --- a/ballista/rust/core/src/serde/physical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/to_proto.rs @@ -272,6 +272,7 @@ impl TryInto for Arc { has_header: exec.has_header(), delimiter: byte_to_string(exec.delimiter())?, batch_size: exec.batch_size() as u32, + table_partition_dims: exec.table_partition_dims().to_vec(), }, )), }) @@ -298,6 +299,7 @@ impl TryInto for Arc { .map(|n| *n as u32) .collect(), batch_size: exec.batch_size() as u32, + table_partition_dims: exec.table_partition_dims().to_vec(), }, )), }) @@ -328,6 +330,7 @@ impl TryInto for Arc { .collect(), schema: Some(exec.file_schema().as_ref().into()), batch_size: exec.batch_size() as u32, + table_partition_dims: exec.table_partition_dims().to_vec(), }, )), }) diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index bfe87efb9f74..8b4fd3e41cac 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -496,7 +496,7 @@ fn get_table( file_extension: extension.to_owned(), target_partitions, collect_stat: true, - partitions: vec![], + table_partition_dims: vec![], }; Ok(Arc::new(ListingTable::new( diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index 8a90221c4d91..317658f5aa69 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -29,6 +29,7 @@ use super::{FileFormat, PhysicalPlanConfig}; use crate::avro_to_arrow::read_avro_schema_from_reader; use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::Result; +use crate::logical_plan::Expr; use crate::physical_plan::file_format::AvroExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -61,16 +62,9 @@ impl FileFormat for AvroFormat { async fn create_physical_plan( &self, conf: PhysicalPlanConfig, + _filters: &[Expr], ) -> Result> { - let exec = AvroExec::new( - conf.object_store, - conf.files, - conf.statistics, - conf.schema, - conf.projection, - conf.batch_size, - conf.limit, - ); + let exec = AvroExec::new(conf); Ok(Arc::new(exec)) } } @@ -346,7 +340,7 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/{}", testdata, file_name); let format = AvroFormat {}; - let schema = format + let file_schema = format .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); @@ -354,18 +348,21 @@ mod tests { .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![local_unpartitioned_file(filename.to_owned())]]; + let file_groups = vec![vec![local_unpartitioned_file(filename.to_owned())]]; let exec = format - .create_physical_plan(PhysicalPlanConfig { - object_store: Arc::new(LocalFileSystem {}), - schema, - files, - statistics, - projection: projection.clone(), - batch_size, - filters: vec![], - limit, - }) + .create_physical_plan( + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema, + file_groups, + statistics, + projection: projection.clone(), + batch_size, + limit, + table_partition_dims: vec![], + }, + &[], + ) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index 6e2654852046..885acf23de5a 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -28,6 +28,7 @@ use futures::StreamExt; use super::{FileFormat, PhysicalPlanConfig}; use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::Result; +use crate::logical_plan::Expr; use crate::physical_plan::file_format::CsvExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -123,18 +124,9 @@ impl FileFormat for CsvFormat { async fn create_physical_plan( &self, conf: PhysicalPlanConfig, + _filters: &[Expr], ) -> Result> { - let exec = CsvExec::new( - conf.object_store, - conf.files, - conf.statistics, - conf.schema, - self.has_header, - self.delimiter, - conf.projection, - conf.batch_size, - conf.limit, - ); + let exec = CsvExec::new(conf, self.has_header, self.delimiter); Ok(Arc::new(exec)) } } @@ -260,7 +252,7 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/csv/{}", testdata, file_name); let format = CsvFormat::default(); - let schema = format + let file_schema = format .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); @@ -268,18 +260,21 @@ mod tests { .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![local_unpartitioned_file(filename.to_owned())]]; + let file_groups = vec![vec![local_unpartitioned_file(filename.to_owned())]]; let exec = format - .create_physical_plan(PhysicalPlanConfig { - object_store: Arc::new(LocalFileSystem {}), - schema, - files, - statistics, - projection: projection.clone(), - batch_size, - filters: vec![], - limit, - }) + .create_physical_plan( + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema, + file_groups, + statistics, + projection: projection.clone(), + batch_size, + limit, + table_partition_dims: vec![], + }, + &[], + ) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index c0ce978d51de..372c5b99c154 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -32,6 +32,7 @@ use super::FileFormat; use super::PhysicalPlanConfig; use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::Result; +use crate::logical_plan::Expr; use crate::physical_plan::file_format::NdJsonExec; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -93,16 +94,9 @@ impl FileFormat for JsonFormat { async fn create_physical_plan( &self, conf: PhysicalPlanConfig, + _filters: &[Expr], ) -> Result> { - let exec = NdJsonExec::new( - conf.object_store, - conf.files, - conf.statistics, - conf.schema, - conf.projection, - conf.batch_size, - conf.limit, - ); + let exec = NdJsonExec::new(conf); Ok(Arc::new(exec)) } } @@ -211,7 +205,7 @@ mod tests { ) -> Result> { let filename = "tests/jsons/2.json"; let format = JsonFormat::default(); - let schema = format + let file_schema = format .infer_schema(local_object_reader_stream(vec![filename.to_owned()])) .await .expect("Schema inference"); @@ -219,18 +213,21 @@ mod tests { .infer_stats(local_object_reader(filename.to_owned())) .await .expect("Stats inference"); - let files = vec![vec![local_unpartitioned_file(filename.to_owned())]]; + let file_groups = vec![vec![local_unpartitioned_file(filename.to_owned())]]; let exec = format - .create_physical_plan(PhysicalPlanConfig { - object_store: Arc::new(LocalFileSystem {}), - schema, - files, - statistics, - projection: projection.clone(), - batch_size, - filters: vec![], - limit, - }) + .create_physical_plan( + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema, + file_groups, + statistics, + projection: projection.clone(), + batch_size, + limit, + table_partition_dims: vec![], + }, + &[], + ) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index d545596f6e5c..83d30b5e938a 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -36,25 +36,25 @@ use async_trait::async_trait; use super::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; use super::PartitionedFile; -/// The configurations to be passed when creating a physical plan for -/// a given file format. +/// The base configurations to provide when creating a physical plan for +/// any given file format. pub struct PhysicalPlanConfig { /// Store from which the `files` should be fetched pub object_store: Arc, /// Schema before projection - pub schema: SchemaRef, + pub file_schema: SchemaRef, /// List of files to be processed, grouped into partitions - pub files: Vec>, + pub file_groups: Vec>, /// Estimated overall statistics of the plan, taking `filters` into account pub statistics: Statistics, /// Columns on which to project the data pub projection: Option>, /// The maximum number of records per arrow column pub batch_size: usize, - /// The filters that were pushed down to this execution plan - pub filters: Vec, /// The minimum number of records required from this source plan pub limit: Option, + /// The partitioning column names + pub table_partition_dims: Vec, } /// This trait abstracts all the file format specific implementations @@ -81,5 +81,6 @@ pub trait FileFormat: Send + Sync + fmt::Debug { async fn create_physical_plan( &self, conf: PhysicalPlanConfig, + filters: &[Expr], ) -> Result>; } diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index 8909fb93a7db..c1d9cb45a67e 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -42,6 +42,7 @@ use crate::datasource::{create_max_min_accs, get_col_stats}; use crate::error::DataFusionError; use crate::error::Result; use crate::logical_plan::combine_filters; +use crate::logical_plan::Expr; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::file_format::ParquetExec; use crate::physical_plan::ExecutionPlan; @@ -104,26 +105,18 @@ impl FileFormat for ParquetFormat { async fn create_physical_plan( &self, conf: PhysicalPlanConfig, + filters: &[Expr], ) -> Result> { // If enable pruning then combine the filters to build the predicate. // If disable pruning then set the predicate to None, thus readers // will not prune data based on the statistics. let predicate = if self.enable_pruning { - combine_filters(&conf.filters) + combine_filters(filters) } else { None }; - Ok(Arc::new(ParquetExec::new( - conf.object_store, - conf.files, - conf.statistics, - conf.schema, - conf.projection, - predicate, - conf.batch_size, - conf.limit, - ))) + Ok(Arc::new(ParquetExec::new(conf, predicate))) } } @@ -592,7 +585,7 @@ mod tests { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/{}", testdata, file_name); let format = ParquetFormat::default(); - let schema = format + let file_schema = format .infer_schema(local_object_reader_stream(vec![filename.clone()])) .await .expect("Schema inference"); @@ -600,18 +593,21 @@ mod tests { .infer_stats(local_object_reader(filename.clone())) .await .expect("Stats inference"); - let files = vec![vec![local_unpartitioned_file(filename.clone())]]; + let file_groups = vec![vec![local_unpartitioned_file(filename.clone())]]; let exec = format - .create_physical_plan(PhysicalPlanConfig { - object_store: Arc::new(LocalFileSystem {}), - schema, - files, - statistics, - projection: projection.clone(), - batch_size, - filters: vec![], - limit, - }) + .create_physical_plan( + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema, + file_groups, + statistics, + projection: projection.clone(), + batch_size, + limit, + table_partition_dims: vec![], + }, + &[], + ) .await?; Ok(exec) } diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs index 34ddb66bea75..17c6a44bafa2 100644 --- a/datafusion/src/datasource/listing/helpers.rs +++ b/datafusion/src/datasource/listing/helpers.rs @@ -29,7 +29,7 @@ use arrow::{ }; use chrono::{TimeZone, Utc}; use futures::{ - stream::{self, TryChunksError}, + stream::{self}, StreamExt, TryStreamExt, }; use log::debug; @@ -79,9 +79,9 @@ pub async fn pruned_partition_list( table_path: &str, filters: &[Expr], file_extension: &str, - partition_names: &[String], + table_partition_dims: &[String], ) -> Result { - if partition_names.is_empty() { + if table_partition_dims.is_empty() { Ok(Box::pin( store .list_file_with_suffix(table_path, file_extension) @@ -96,17 +96,19 @@ pub async fn pruned_partition_list( } else { let applicable_filters = filters .iter() - .filter(|f| expr_applicable_for_cols(partition_names, f)); + .filter(|f| expr_applicable_for_cols(table_partition_dims, f)); - let partition_names = partition_names.to_vec(); + let table_partition_dims = table_partition_dims.to_vec(); let stream_path = table_path.to_owned(); // TODO avoid collecting but have a streaming memory table instead let batches: Vec = store .list_file_with_suffix(table_path, file_extension) .await? - .try_chunks(64) - .map_err(|TryChunksError(_, e)| e) - .map(move |metas| paths_to_batch(&partition_names, &stream_path, &metas?)) + .chunks(64) + .map(|v| v.into_iter().collect::>>()) + .map(move |metas| { + paths_to_batch(&table_partition_dims, &stream_path, &metas?) + }) .try_collect() .await?; @@ -135,20 +137,20 @@ pub async fn pruned_partition_list( /// - ... one column by partition ... /// Note: For the last modified date, this looses precisions higher than millisecond. fn paths_to_batch( - partition_names: &[String], + table_partition_dims: &[String], table_path: &str, metas: &[FileMeta], ) -> Result { let mut key_builder = StringBuilder::new(metas.len()); let mut length_builder = UInt64Builder::new(metas.len()); let mut modified_builder = Date64Builder::new(metas.len()); - let mut partition_builders = partition_names + let mut partition_builders = table_partition_dims .iter() .map(|_| StringBuilder::new(metas.len())) .collect::>(); for file_meta in metas { if let Some(partition_values) = - parse_partitions_for_path(table_path, file_meta.path(), partition_names) + parse_partitions_for_path(table_path, file_meta.path(), table_partition_dims) { key_builder.append_value(file_meta.path())?; length_builder.append_value(file_meta.size())?; @@ -180,7 +182,7 @@ fn paths_to_batch( Field::new(FILE_PATH_COLUMN_NAME, DataType::UInt64, false), Field::new(FILE_MODIFIED_COLUMN_NAME, DataType::Date64, false), ]; - for pn in partition_names { + for pn in table_partition_dims { fields.push(Field::new(pn, DataType::Utf8, false)); } @@ -231,11 +233,11 @@ fn batches_to_paths(batches: &[RecordBatch]) -> Vec { } /// Extract the partition values for the given `file_path` (in the given `table_path`) -/// associated to the partitions defined by `partition_names` +/// associated to the partitions defined by `table_partition_dims` fn parse_partitions_for_path<'a>( table_path: &str, file_path: &'a str, - partition_names: &[String], + table_partition_dims: &[String], ) -> Option> { let subpath = file_path.strip_prefix(table_path)?; @@ -246,7 +248,7 @@ fn parse_partitions_for_path<'a>( }; let mut part_values = vec![]; - for (path, pn) in subpath.split('/').zip(partition_names) { + for (path, pn) in subpath.split('/').zip(table_partition_dims) { if let Some(val) = path.strip_prefix(&format!("{}=", pn)) { part_values.push(val); } else { diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index 3c512ee955a5..8e96d5e54baa 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -52,7 +52,7 @@ pub struct ListingOptions { /// - If there is a third level of partitioning it will be ignored. /// - Files that don't follow this partitioning will be ignored. /// Note that only `DataType::Utf8` is supported for the column type. - pub partitions: Vec, + pub table_partition_dims: Vec, /// 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. @@ -73,7 +73,7 @@ impl ListingOptions { Self { file_extension: String::new(), format, - partitions: vec![], + table_partition_dims: vec![], collect_stat: false, target_partitions: 1, } @@ -97,7 +97,7 @@ impl ListingOptions { let file_schema = self.format.infer_schema(Box::pin(file_stream)).await?; // Add the partition columns to the file schema let mut fields = file_schema.fields().clone(); - for part in &self.partitions { + for part in &self.table_partition_dims { fields.push(Field::new(part, DataType::Utf8, false)); } Ok(Arc::new(Schema::new(fields))) @@ -180,16 +180,19 @@ impl TableProvider for ListingTable { // create the execution plan self.options .format - .create_physical_plan(PhysicalPlanConfig { - object_store: Arc::clone(&self.object_store), - schema: self.schema(), - files: partitioned_file_lists, - statistics, - projection: projection.clone(), - batch_size, - filters: filters.to_vec(), - limit, - }) + .create_physical_plan( + PhysicalPlanConfig { + object_store: Arc::clone(&self.object_store), + file_schema: self.schema(), + file_groups: partitioned_file_lists, + statistics, + projection: projection.clone(), + batch_size, + limit, + table_partition_dims: self.options.table_partition_dims.clone(), + }, + filters, + ) .await } @@ -215,7 +218,7 @@ impl ListingTable { &self.table_path, filters, &self.options.file_extension, - &self.options.partitions, + &self.options.table_partition_dims, ) .await?; @@ -285,7 +288,7 @@ mod tests { let opt = ListingOptions { file_extension: ".avro".to_owned(), format: Arc::new(AvroFormat {}), - partitions: vec![String::from("p1")], + table_partition_dims: vec![String::from("p1")], target_partitions: 4, collect_stat: true, }; @@ -377,7 +380,7 @@ mod tests { let opt = ListingOptions { file_extension: "parquet".to_owned(), format: Arc::new(ParquetFormat::default()), - partitions: vec![], + table_partition_dims: vec![], target_partitions: 2, collect_stat: true, }; @@ -407,7 +410,7 @@ mod tests { let opt = ListingOptions { file_extension: "".to_owned(), format: Arc::new(format), - partitions: vec![], + table_partition_dims: vec![], target_partitions, collect_stat: true, }; diff --git a/datafusion/src/datasource/mod.rs b/datafusion/src/datasource/mod.rs index 6097a21a52c1..efea47b1d437 100644 --- a/datafusion/src/datasource/mod.rs +++ b/datafusion/src/datasource/mod.rs @@ -41,6 +41,7 @@ use std::pin::Pin; /// if the optional `limit` is provided, includes only sufficient files /// needed to read up to `limit` number of rows /// TODO fix case where `num_rows` and `total_byte_size` are not defined (stat should be None instead of Some(0)) +/// TODO check that stats for partition columns are correct pub async fn get_statistics_with_limit( all_files: impl Stream>, schema: SchemaRef, diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index bb23b3f4e91c..2dca98747b00 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -221,7 +221,7 @@ impl ExecutionContext { .unwrap() .config .target_partitions, - partitions: vec![], + table_partition_dims: vec![], }; // TODO make schema in CreateExternalTable optional instead of empty @@ -442,7 +442,7 @@ impl ExecutionContext { collect_stat: true, file_extension: DEFAULT_PARQUET_EXTENSION.to_owned(), target_partitions, - partitions: vec![], + table_partition_dims: vec![], }; self.register_listing_table(name, uri, listing_options, None) diff --git a/datafusion/src/execution/options.rs b/datafusion/src/execution/options.rs index f0ed6f24c325..255cf9285240 100644 --- a/datafusion/src/execution/options.rs +++ b/datafusion/src/execution/options.rs @@ -108,7 +108,7 @@ impl<'a> CsvReadOptions<'a> { collect_stat: false, file_extension: self.file_extension.to_owned(), target_partitions, - partitions: vec![], + table_partition_dims: vec![], } } } @@ -143,7 +143,7 @@ impl<'a> AvroReadOptions<'a> { collect_stat: false, file_extension: self.file_extension.to_owned(), target_partitions, - partitions: vec![], + table_partition_dims: vec![], } } } diff --git a/datafusion/src/logical_plan/builder.rs b/datafusion/src/logical_plan/builder.rs index 09c3a14513e5..6865ec0da287 100644 --- a/datafusion/src/logical_plan/builder.rs +++ b/datafusion/src/logical_plan/builder.rs @@ -277,7 +277,7 @@ impl LogicalPlanBuilder { collect_stat: true, file_extension: DEFAULT_PARQUET_EXTENSION.to_owned(), target_partitions, - partitions: vec![], + table_partition_dims: vec![], }; let path: String = path.into(); diff --git a/datafusion/src/physical_optimizer/repartition.rs b/datafusion/src/physical_optimizer/repartition.rs index ea7de7f39839..afd4af5c7f22 100644 --- a/datafusion/src/physical_optimizer/repartition.rs +++ b/datafusion/src/physical_optimizer/repartition.rs @@ -109,7 +109,7 @@ mod tests { use arrow::datatypes::Schema; use super::*; - use crate::datasource::object_store::local::LocalFileSystem; + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::PartitionedFile; use crate::physical_plan::file_format::ParquetExec; use crate::physical_plan::projection::ProjectionExec; @@ -118,17 +118,20 @@ mod tests { #[test] fn added_repartition_to_single_partition() -> Result<()> { - let schema = Arc::new(Schema::empty()); + let file_schema = Arc::new(Schema::empty()); let parquet_project = ProjectionExec::try_new( vec![], Arc::new(ParquetExec::new( - TestObjectStore::new_arc(&[("x", 100)]), - vec![vec![PartitionedFile::new("x".to_string(), 100)]], - Statistics::default(), - schema, - None, - None, - 2048, + PhysicalPlanConfig { + object_store: TestObjectStore::new_arc(&[("x", 100)]), + file_schema, + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::default(), + projection: None, + batch_size: 2048, + limit: None, + table_partition_dims: vec![], + }, None, )), )?; @@ -152,19 +155,25 @@ mod tests { #[test] fn repartition_deepest_node() -> Result<()> { - let schema = Arc::new(Schema::empty()); + let file_schema = Arc::new(Schema::empty()); let parquet_project = ProjectionExec::try_new( vec![], Arc::new(ProjectionExec::try_new( vec![], Arc::new(ParquetExec::new( - Arc::new(LocalFileSystem {}), - vec![vec![PartitionedFile::new("x".to_string(), 100)]], - Statistics::default(), - schema, - None, - None, - 2048, + PhysicalPlanConfig { + object_store: TestObjectStore::new_arc(&[("x", 100)]), + file_schema, + file_groups: vec![vec![PartitionedFile::new( + "x".to_string(), + 100, + )]], + statistics: Statistics::default(), + projection: None, + batch_size: 2048, + limit: None, + table_partition_dims: vec![], + }, None, )), )?), diff --git a/datafusion/src/physical_plan/coalesce_partitions.rs b/datafusion/src/physical_plan/coalesce_partitions.rs index 1fd18d2c4f37..94198aec2afe 100644 --- a/datafusion/src/physical_plan/coalesce_partitions.rs +++ b/datafusion/src/physical_plan/coalesce_partitions.rs @@ -206,6 +206,7 @@ mod tests { use futures::FutureExt; use super::*; + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::file_format::CsvExec; use crate::physical_plan::{collect, common}; @@ -220,15 +221,18 @@ mod tests { let (_, files) = test::create_partitioned_csv("aggregate_test_100.csv", num_partitions)?; let csv = CsvExec::new( - Arc::new(LocalFileSystem {}), - files, - Statistics::default(), - schema, + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: schema, + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); // input should have 4 partitions diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index 9bae1ebd59b8..076446bd046d 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -18,13 +18,14 @@ //! Execution plan for reading line-delimited Avro files #[cfg(feature = "avro")] use crate::avro_to_arrow; +use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::ObjectStore; use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -use arrow::datatypes::{Schema, SchemaRef}; +use arrow::datatypes::SchemaRef; #[cfg(feature = "avro")] use arrow::error::ArrowError; @@ -46,35 +47,28 @@ pub struct AvroExec { projected_schema: SchemaRef, batch_size: usize, limit: Option, + table_partition_dims: Vec, } impl AvroExec { /// Create a new Avro reader execution plan provided file list and schema - pub fn new( - object_store: Arc, - file_groups: Vec>, - statistics: Statistics, - file_schema: SchemaRef, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Self { - let projected_schema = match &projection { - None => Arc::clone(&file_schema), - Some(p) => Arc::new(Schema::new( - p.iter().map(|i| file_schema.field(*i).clone()).collect(), - )), - }; + pub fn new(base_config: PhysicalPlanConfig) -> Self { + let (projected_schema, projected_statistics) = super::project( + &base_config.projection, + Arc::clone(&base_config.file_schema), + base_config.statistics, + ); Self { - object_store, - file_groups, - statistics, - file_schema, - projection, + object_store: base_config.object_store, + file_groups: base_config.file_groups, + statistics: projected_statistics, + file_schema: base_config.file_schema, + projection: base_config.projection, projected_schema, - batch_size, - limit, + batch_size: base_config.batch_size, + limit: base_config.limit, + table_partition_dims: base_config.table_partition_dims, } } /// List of data files @@ -97,6 +91,10 @@ impl AvroExec { pub fn limit(&self) -> Option { self.limit } + /// Partitioning column names + pub fn table_partition_dims(&self) -> &[String] { + &self.table_partition_dims + } } #[async_trait] @@ -172,6 +170,7 @@ impl ExecutionPlan for AvroExec { fun, Arc::clone(&self.projected_schema), self.limit, + self.table_partition_dims.clone(), ))) } @@ -216,17 +215,18 @@ mod tests { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); - let avro_exec = AvroExec::new( - Arc::new(LocalFileSystem {}), - vec![vec![local_unpartitioned_file(filename.clone())]], - Statistics::default(), - AvroFormat {} + let avro_exec = AvroExec::new(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_groups: vec![vec![local_unpartitioned_file(filename.clone())]], + file_schema: AvroFormat {} .infer_schema(local_object_reader_stream(vec![filename])) .await?, - Some(vec![0, 1, 2]), - 1024, - None, - ); + statistics: Statistics::default(), + projection: Some(vec![0, 1, 2]), + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }); assert_eq!(avro_exec.output_partitioning().partition_count(), 1); let mut results = avro_exec.execute(0).await?; diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index 3347198de4ba..77590d6fae5c 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -17,6 +17,7 @@ //! Execution plan for reading CSV files +use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::ObjectStore; use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; @@ -25,7 +26,7 @@ use crate::physical_plan::{ }; use arrow::csv; -use arrow::datatypes::{Schema, SchemaRef}; +use arrow::datatypes::SchemaRef; use std::any::Any; use std::sync::Arc; @@ -48,40 +49,30 @@ pub struct CsvExec { projection: Option>, batch_size: usize, limit: Option, + table_partition_dims: Vec, } impl CsvExec { /// Create a new CSV reader execution plan provided file list and schema - #[allow(clippy::too_many_arguments)] - pub fn new( - object_store: Arc, - file_groups: Vec>, - statistics: Statistics, - file_schema: SchemaRef, - has_header: bool, - delimiter: u8, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Self { - let projected_schema = match &projection { - None => Arc::clone(&file_schema), - Some(p) => Arc::new(Schema::new( - p.iter().map(|i| file_schema.field(*i).clone()).collect(), - )), - }; + pub fn new(base_config: PhysicalPlanConfig, has_header: bool, delimiter: u8) -> Self { + let (projected_schema, projected_statistics) = super::project( + &base_config.projection, + Arc::clone(&base_config.file_schema), + base_config.statistics, + ); Self { - object_store, - file_groups, - file_schema, - statistics, + object_store: base_config.object_store, + file_groups: base_config.file_groups, + file_schema: base_config.file_schema, + statistics: projected_statistics, has_header, delimiter, - projection, + projection: base_config.projection, projected_schema, - batch_size, - limit, + batch_size: base_config.batch_size, + limit: base_config.limit, + table_partition_dims: base_config.table_partition_dims, } } @@ -113,6 +104,10 @@ impl CsvExec { pub fn limit(&self) -> Option { self.limit } + /// Partitioning column names + pub fn table_partition_dims(&self) -> &[String] { + &self.table_partition_dims + } } #[async_trait] @@ -178,6 +173,7 @@ impl ExecutionPlan for CsvExec { fun, Arc::clone(&self.projected_schema), self.limit, + self.table_partition_dims.clone(), ))) } @@ -216,20 +212,23 @@ mod tests { #[tokio::test] async fn csv_exec_with_projection() -> Result<()> { - let schema = aggr_test_schema(); + let file_schema = aggr_test_schema(); let testdata = crate::test_util::arrow_test_data(); let filename = "aggregate_test_100.csv"; let path = format!("{}/csv/{}", testdata, filename); let csv = CsvExec::new( - Arc::new(LocalFileSystem {}), - vec![vec![local_unpartitioned_file(path)]], - Statistics::default(), - schema, + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema, + file_groups: vec![vec![local_unpartitioned_file(path)]], + statistics: Statistics::default(), + projection: Some(vec![0, 2, 4]), + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - Some(vec![0, 2, 4]), - 1024, - None, ); assert_eq!(13, csv.file_schema.fields().len()); assert_eq!(3, csv.projected_schema.fields().len()); @@ -247,20 +246,23 @@ mod tests { #[tokio::test] async fn csv_exec_without_projection() -> Result<()> { - let schema = aggr_test_schema(); + let file_schema = aggr_test_schema(); let testdata = crate::test_util::arrow_test_data(); let filename = "aggregate_test_100.csv"; let path = format!("{}/csv/{}", testdata, filename); let csv = CsvExec::new( - Arc::new(LocalFileSystem {}), - vec![vec![local_unpartitioned_file(path)]], - Statistics::default(), - schema, + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema, + file_groups: vec![vec![local_unpartitioned_file(path)]], + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); assert_eq!(13, csv.file_schema.fields().len()); assert_eq!(13, csv.projected_schema.fields().len()); diff --git a/datafusion/src/physical_plan/file_format/file_stream.rs b/datafusion/src/physical_plan/file_format/file_stream.rs index 55a66f46cf48..5863a136ea8b 100644 --- a/datafusion/src/physical_plan/file_format/file_stream.rs +++ b/datafusion/src/physical_plan/file_format/file_stream.rs @@ -23,16 +23,19 @@ use crate::{ datasource::{object_store::ObjectStore, PartitionedFile}, - error::Result as DataFusionResult, physical_plan::RecordBatchStream, + scalar::ScalarValue, }; use arrow::{ - datatypes::SchemaRef, + array::{ArrayData, ArrayRef, DictionaryArray, UInt8BufferBuilder}, + buffer::Buffer, + datatypes::{DataType, SchemaRef, UInt8Type}, error::{ArrowError, Result as ArrowResult}, record_batch::RecordBatch, }; use futures::Stream; use std::{ + collections::HashMap, io::Read, iter, pin::Pin, @@ -40,8 +43,7 @@ use std::{ task::{Context, Poll}, }; -pub type FileIter = - Box>> + Send + Sync>; +pub type FileIter = Box + Send + Sync>; pub type BatchIter = Box> + Send + Sync>; /// A closure that creates a file format reader (iterator over `RecordBatch`) from a `Read` object @@ -63,9 +65,12 @@ impl FormatReaderOpener for T where pub struct FileStream { /// An iterator over record batches of the last file returned by file_iter batch_iter: BatchIter, - /// An iterator over input files + /// Partitioning column values for the current batch_iter + partition_values: Vec, + /// An iterator over input files. file_iter: FileIter, - /// The stream schema (file schema after projection) + /// The stream schema (file schema including partition columns and after + /// projection). schema: SchemaRef, /// The remaining number of records to parse, None if no limit remain: Option, @@ -74,6 +79,15 @@ pub struct FileStream { /// is not capable of limiting the number of records in the last batch, the file /// stream will take care of truncating it. file_reader: F, + /// A buffer initialized to zeros that represents the key array of all partition + /// columns (partition columns are materialized by dictionary arrays with only one + /// value in the dictionary, thus all the keys are equal to zero). + key_buffer_cache: Option, + /// mapping between the indexes in the list of partition columns and the target + /// schema. + projected_partition_indexes: HashMap, + /// the store from which to source the files. + object_store: Arc, } impl FileStream { @@ -83,38 +97,103 @@ impl FileStream { file_reader: F, schema: SchemaRef, limit: Option, + table_partition_dims: Vec, ) -> Self { - let read_iter = files.into_iter().map(move |f| -> DataFusionResult<_> { - object_store - .file_reader(f.file_meta.sized_file)? - .sync_reader() - }); + let mut projected_partition_indexes = HashMap::new(); + for (partition_idx, partition_name) in table_partition_dims.iter().enumerate() { + if let Ok(schema_idx) = schema.index_of(partition_name) { + projected_partition_indexes.insert(partition_idx, schema_idx); + } + } Self { - file_iter: Box::new(read_iter), + file_iter: Box::new(files.into_iter()), batch_iter: Box::new(iter::empty()), + partition_values: vec![], remain: limit, schema, file_reader, + key_buffer_cache: None, + projected_partition_indexes, + object_store, } } - /// Acts as a flat_map of record batches over files. + /// Acts as a flat_map of record batches over files. Adds the partitioning + /// Columns to the returned record batches. fn next_batch(&mut self) -> Option> { + let expected_cols = + self.schema.fields().len() - self.projected_partition_indexes.len(); match self.batch_iter.next() { - Some(batch) => Some(batch), + Some(Ok(batch)) if batch.columns().len() == expected_cols => { + let mut cols = batch.columns().to_vec(); + for (&pidx, &sidx) in &self.projected_partition_indexes { + cols.insert( + sidx, + create_dict_array( + &mut self.key_buffer_cache, + &self.partition_values[pidx], + batch.num_rows(), + ), + ) + } + Some(RecordBatch::try_new(self.schema(), cols)) + } + Some(Ok(batch)) => Some(Err(ArrowError::SchemaError(format!( + "Unexpected batch schema from file, expected {} cols but got {}", + expected_cols, + batch.columns().len() + )))), + Some(Err(e)) => Some(Err(e)), None => match self.file_iter.next() { - Some(Ok(f)) => { - self.batch_iter = (self.file_reader)(f, &self.remain); - self.next_batch() + Some(f) => { + self.partition_values = f.partition_values; + self.object_store + .file_reader(f.file_meta.sized_file) + .and_then(|r| r.sync_reader()) + .map_err(|e| ArrowError::ExternalError(Box::new(e))) + .and_then(|f| { + self.batch_iter = (self.file_reader)(f, &self.remain); + self.next_batch().transpose() + }) + .transpose() } - Some(Err(e)) => Some(Err(ArrowError::ExternalError(Box::new(e)))), None => None, }, } } } +fn create_dict_array( + key_buffer_cache: &mut Option, + val: &ScalarValue, + len: usize, +) -> ArrayRef { + // build value dictionary + let dict_vals = val.to_array(); + + // build keys array + let sliced_key_buffer = match key_buffer_cache { + Some(buf) if buf.len() >= len => buf.slice(buf.len() - len), + _ => { + let mut key_buffer_builder = UInt8BufferBuilder::new(len); + key_buffer_builder.advance(len); // keys are all 0 + key_buffer_cache.insert(key_buffer_builder.finish()).clone() + } + }; + + // create data type + let data_type = + DataType::Dictionary(Box::new(DataType::UInt8), Box::new(val.get_datatype())); + + // assemble pieces together + let mut builder = ArrayData::builder(data_type) + .len(len) + .add_buffer(sliced_key_buffer); + builder = builder.add_child_data(dict_vals.data().clone()); + Arc::new(DictionaryArray::::from(builder.build().unwrap())) +} + impl Stream for FileStream { type Item = ArrowResult; @@ -191,6 +270,7 @@ mod tests { reader, source_schema, limit, + vec![], ); file_stream diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index c1ee0cb87c56..a705e9d70f7d 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -18,16 +18,14 @@ //! Execution plan for reading line-delimited JSON files use async_trait::async_trait; +use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::ObjectStore; use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -use arrow::{ - datatypes::{Schema, SchemaRef}, - json, -}; +use arrow::{datatypes::SchemaRef, json}; use std::any::Any; use std::sync::Arc; @@ -44,35 +42,28 @@ pub struct NdJsonExec { projected_schema: SchemaRef, batch_size: usize, limit: Option, + table_partition_dims: Vec, } impl NdJsonExec { /// Create a new JSON reader execution plan provided file list and schema - pub fn new( - object_store: Arc, - file_groups: Vec>, - statistics: Statistics, - file_schema: SchemaRef, - projection: Option>, - batch_size: usize, - limit: Option, - ) -> Self { - let projected_schema = match &projection { - None => Arc::clone(&file_schema), - Some(p) => Arc::new(Schema::new( - p.iter().map(|i| file_schema.field(*i).clone()).collect(), - )), - }; + pub fn new(base_config: PhysicalPlanConfig) -> Self { + let (projected_schema, projected_statistics) = super::project( + &base_config.projection, + Arc::clone(&base_config.file_schema), + base_config.statistics, + ); Self { - object_store, - file_groups, - statistics, - file_schema, - projection, + object_store: base_config.object_store, + file_groups: base_config.file_groups, + statistics: projected_statistics, + file_schema: base_config.file_schema, + projection: base_config.projection, projected_schema, - batch_size, - limit, + batch_size: base_config.batch_size, + limit: base_config.limit, + table_partition_dims: base_config.table_partition_dims, } } } @@ -136,6 +127,7 @@ impl ExecutionPlan for NdJsonExec { fun, Arc::clone(&self.projected_schema), self.limit, + self.table_partition_dims.clone(), ))) } @@ -187,15 +179,16 @@ mod tests { async fn nd_json_exec_file_without_projection() -> Result<()> { use arrow::datatypes::DataType; let path = format!("{}/1.json", TEST_DATA_BASE); - let exec = NdJsonExec::new( - Arc::new(LocalFileSystem {}), - vec![vec![local_unpartitioned_file(path.clone())]], - Default::default(), - infer_schema(path).await?, - None, - 1024, - Some(3), - ); + let exec = NdJsonExec::new(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_groups: vec![vec![local_unpartitioned_file(path.clone())]], + file_schema: infer_schema(path).await?, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: Some(3), + table_partition_dims: vec![], + }); // TODO: this is not where schema inference should be tested @@ -240,15 +233,16 @@ mod tests { #[tokio::test] async fn nd_json_exec_file_projection() -> Result<()> { let path = format!("{}/1.json", TEST_DATA_BASE); - let exec = NdJsonExec::new( - Arc::new(LocalFileSystem {}), - vec![vec![local_unpartitioned_file(path.clone())]], - Default::default(), - infer_schema(path).await?, - Some(vec![0, 2]), - 1024, - None, - ); + let exec = NdJsonExec::new(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_groups: vec![vec![local_unpartitioned_file(path.clone())]], + file_schema: infer_schema(path).await?, + statistics: Statistics::default(), + projection: Some(vec![0, 2]), + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 2); diff --git a/datafusion/src/physical_plan/file_format/mod.rs b/datafusion/src/physical_plan/file_format/mod.rs index b0b690519eca..be84151201c9 100644 --- a/datafusion/src/physical_plan/file_format/mod.rs +++ b/datafusion/src/physical_plan/file_format/mod.rs @@ -24,19 +24,25 @@ mod json; mod parquet; pub use self::parquet::ParquetExec; +use arrow::datatypes::{Schema, SchemaRef}; pub use avro::AvroExec; pub use csv::CsvExec; pub use json::NdJsonExec; use crate::datasource::PartitionedFile; -use std::fmt::{Display, Formatter, Result}; +use std::{ + fmt::{Display, Formatter, Result as FmtResult}, + sync::Arc, +}; + +use super::Statistics; /// A wrapper to customize partitioned file display #[derive(Debug)] struct FileGroupsDisplay<'a>(&'a [Vec]); impl<'a> Display for FileGroupsDisplay<'a> { - fn fmt(&self, f: &mut Formatter) -> Result { + fn fmt(&self, f: &mut Formatter) -> FmtResult { let parts: Vec<_> = self .0 .iter() @@ -50,3 +56,39 @@ impl<'a> Display for FileGroupsDisplay<'a> { write!(f, "[{}]", parts.join(", ")) } } + +/// Project the schema and the statistics on the given column indices +fn project( + projection: &Option>, + schema: SchemaRef, + statistics: Statistics, +) -> (SchemaRef, Statistics) { + let projection = if projection.is_none() { + return (schema, statistics); + } else { + projection.as_ref().unwrap() + }; + let projected_schema = Schema::new( + projection + .iter() + .map(|i| schema.field(*i).clone()) + .collect(), + ); + + let new_column_statistics = statistics.column_statistics.map(|stats| { + let mut projected_stats = Vec::with_capacity(projection.len()); + for proj in projection { + projected_stats.push(stats[*proj].clone()); + } + projected_stats + }); + + let statistics = Statistics { + num_rows: statistics.num_rows, + total_byte_size: statistics.total_byte_size, + column_statistics: new_column_statistics, + is_exact: statistics.is_exact, + }; + + (Arc::new(projected_schema), statistics) +} diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index b86ed1dc43f8..550af996f1d7 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use std::{any::Any, convert::TryInto}; use crate::datasource::file_format::parquet::ChunkObjectReader; +use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::ObjectStore; use crate::datasource::PartitionedFile; use crate::{ @@ -81,6 +82,8 @@ pub struct ParquetExec { predicate_builder: Option, /// Optional limit of the number of rows limit: Option, + /// Partioning column names + table_partition_dims: Vec, } /// Stores metrics about the parquet execution for a particular parquet file @@ -95,26 +98,19 @@ struct ParquetFileMetrics { impl ParquetExec { /// Create a new Parquet reader execution plan provided file list and schema. /// Even if `limit` is set, ParquetExec rounds up the number of records to the next `batch_size`. - #[allow(clippy::too_many_arguments)] - pub fn new( - object_store: Arc, - file_groups: Vec>, - statistics: Statistics, - schema: SchemaRef, - projection: Option>, - predicate: Option, - batch_size: usize, - limit: Option, - ) -> Self { + pub fn new(base_config: PhysicalPlanConfig, predicate: Option) -> Self { debug!("Creating ParquetExec, files: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", - file_groups, projection, predicate, limit); + base_config.file_groups, base_config.projection, predicate, base_config.limit); let metrics = ExecutionPlanMetricsSet::new(); let predicate_creation_errors = MetricBuilder::new(&metrics).global_counter("num_predicate_creation_errors"); let predicate_builder = predicate.and_then(|predicate_expr| { - match PruningPredicate::try_new(&predicate_expr, schema.clone()) { + match PruningPredicate::try_new( + &predicate_expr, + base_config.file_schema.clone(), + ) { Ok(predicate_builder) => Some(predicate_builder), Err(e) => { debug!( @@ -127,57 +123,31 @@ impl ParquetExec { } }); - let projection = match projection { - Some(p) => p, - None => (0..schema.fields().len()).collect(), + let expanded_projection = match &base_config.projection { + Some(p) => p.to_vec(), + None => (0..base_config.file_schema.fields().len()).collect(), }; - let (projected_schema, projected_statistics) = - Self::project(&projection, schema, statistics); + let (projected_schema, projected_statistics) = super::project( + &base_config.projection, + base_config.file_schema, + base_config.statistics, + ); Self { - object_store, - file_groups, + object_store: base_config.object_store, + file_groups: base_config.file_groups, schema: projected_schema, - projection, + projection: expanded_projection, metrics, predicate_builder, - batch_size, + batch_size: base_config.batch_size, statistics: projected_statistics, - limit, + limit: base_config.limit, + table_partition_dims: base_config.table_partition_dims, } } - fn project( - projection: &[usize], - schema: SchemaRef, - statistics: Statistics, - ) -> (SchemaRef, Statistics) { - let projected_schema = Schema::new( - projection - .iter() - .map(|i| schema.field(*i).clone()) - .collect(), - ); - - let new_column_statistics = statistics.column_statistics.map(|stats| { - let mut projected_stats = Vec::with_capacity(projection.len()); - for proj in projection { - projected_stats.push(stats[*proj].clone()); - } - projected_stats - }); - - let statistics = Statistics { - num_rows: statistics.num_rows, - total_byte_size: statistics.total_byte_size, - column_statistics: new_column_statistics, - is_exact: statistics.is_exact, - }; - - (Arc::new(projected_schema), statistics) - } - /// List of data files pub fn file_groups(&self) -> &[Vec] { &self.file_groups @@ -190,11 +160,14 @@ impl ParquetExec { pub fn batch_size(&self) -> usize { self.batch_size } - /// Limit in nr. of rows pub fn limit(&self) -> Option { self.limit } + /// Partitioning column names + pub fn table_partition_dims(&self) -> &[String] { + &self.table_partition_dims + } } impl ParquetFileMetrics { @@ -483,6 +456,7 @@ fn read_partition( match batch_reader.next() { Some(Ok(batch)) => { total_rows += batch.num_rows(); + // TODO add partitioning columns here! send_result(&response_tx, Ok(batch))?; if limit.map(|l| total_rows >= l).unwrap_or(false) { break 'outer; @@ -537,15 +511,18 @@ mod tests { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/alltypes_plain.parquet", testdata); let parquet_exec = ParquetExec::new( - Arc::new(LocalFileSystem {}), - vec![vec![local_unpartitioned_file(filename.clone())]], - Statistics::default(), - ParquetFormat::default() - .infer_schema(local_object_reader_stream(vec![filename])) - .await?, - Some(vec![0, 1, 2]), - None, - 1024, + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_groups: vec![vec![local_unpartitioned_file(filename.clone())]], + file_schema: ParquetFormat::default() + .infer_schema(local_object_reader_stream(vec![filename])) + .await?, + statistics: Statistics::default(), + projection: Some(vec![0, 1, 2]), + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, None, ); assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); diff --git a/datafusion/src/physical_plan/filter.rs b/datafusion/src/physical_plan/filter.rs index 79b5ebc508f5..1b99b47ed458 100644 --- a/datafusion/src/physical_plan/filter.rs +++ b/datafusion/src/physical_plan/filter.rs @@ -222,12 +222,13 @@ impl RecordBatchStream for FilterExecStream { mod tests { use super::*; + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::expressions::*; use crate::physical_plan::file_format::CsvExec; use crate::physical_plan::ExecutionPlan; use crate::scalar::ScalarValue; - use crate::test::{self, aggr_test_schema}; + use crate::test::{self}; use crate::{logical_plan::Operator, physical_plan::collect}; use std::iter::Iterator; @@ -240,15 +241,18 @@ mod tests { test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; let csv = CsvExec::new( - Arc::new(LocalFileSystem {}), - files, - Statistics::default(), - aggr_test_schema(), + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: Arc::clone(&schema), + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); let predicate: Arc = binary( diff --git a/datafusion/src/physical_plan/limit.rs b/datafusion/src/physical_plan/limit.rs index bd48e4d2e5d4..9fa9270f18d0 100644 --- a/datafusion/src/physical_plan/limit.rs +++ b/datafusion/src/physical_plan/limit.rs @@ -384,6 +384,7 @@ mod tests { use common::collect; use super::*; + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::common; @@ -399,15 +400,18 @@ mod tests { test::create_partitioned_csv("aggregate_test_100.csv", num_partitions)?; let csv = CsvExec::new( - Arc::new(LocalFileSystem {}), - files, - Statistics::default(), - schema, + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: schema, + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); // input should have 4 partitions diff --git a/datafusion/src/physical_plan/projection.rs b/datafusion/src/physical_plan/projection.rs index 794d9a2ec68e..fc3bfd21ad76 100644 --- a/datafusion/src/physical_plan/projection.rs +++ b/datafusion/src/physical_plan/projection.rs @@ -259,11 +259,12 @@ impl RecordBatchStream for ProjectionStream { mod tests { use super::*; + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::expressions::{self, col}; use crate::physical_plan::file_format::CsvExec; use crate::scalar::ScalarValue; - use crate::test::{self, aggr_test_schema}; + use crate::test::{self}; use futures::future; #[tokio::test] @@ -275,15 +276,18 @@ mod tests { test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; let csv = CsvExec::new( - Arc::new(LocalFileSystem {}), - files, - Statistics::default(), - aggr_test_schema(), + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: Arc::clone(&schema), + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); // pick column c1 and name it column c1 in the output schema diff --git a/datafusion/src/physical_plan/sort.rs b/datafusion/src/physical_plan/sort.rs index 499d1f743844..59319e9a1b1f 100644 --- a/datafusion/src/physical_plan/sort.rs +++ b/datafusion/src/physical_plan/sort.rs @@ -310,6 +310,7 @@ impl RecordBatchStream for SortStream { #[cfg(test)] mod tests { use super::*; + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::expressions::col; @@ -317,7 +318,7 @@ mod tests { use crate::physical_plan::{collect, file_format::CsvExec}; use crate::test::assert_is_pending; use crate::test::exec::assert_strong_count_converges_to_zero; - use crate::test::{self, aggr_test_schema, exec::BlockingExec}; + use crate::test::{self, exec::BlockingExec}; use arrow::array::*; use arrow::datatypes::*; use futures::FutureExt; @@ -330,15 +331,18 @@ mod tests { test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; let csv = CsvExec::new( - Arc::new(LocalFileSystem {}), - files, - Statistics::default(), - aggr_test_schema(), + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: Arc::clone(&schema), + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); let sort_exec = Arc::new(SortExec::try_new( diff --git a/datafusion/src/physical_plan/sort_preserving_merge.rs b/datafusion/src/physical_plan/sort_preserving_merge.rs index 5aaf9789f699..7e66c5ad7ae4 100644 --- a/datafusion/src/physical_plan/sort_preserving_merge.rs +++ b/datafusion/src/physical_plan/sort_preserving_merge.rs @@ -658,6 +658,7 @@ impl RecordBatchStream for SortPreservingMergeStream { #[cfg(test)] mod tests { + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::metrics::MetricValue; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; @@ -936,15 +937,18 @@ mod tests { test::create_partitioned_csv("aggregate_test_100.csv", partitions).unwrap(); let csv = Arc::new(CsvExec::new( - Arc::new(LocalFileSystem {}), - files, - Statistics::default(), - Arc::clone(&schema), + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: Arc::clone(&schema), + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, )); let sort = vec![ @@ -1016,15 +1020,18 @@ mod tests { test::create_partitioned_csv("aggregate_test_100.csv", partitions).unwrap(); let csv = Arc::new(CsvExec::new( - Arc::new(LocalFileSystem {}), - files, - Statistics::default(), - schema, + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: schema, + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, )); let sorted = basic_sort(csv, sort).await; diff --git a/datafusion/src/physical_plan/union.rs b/datafusion/src/physical_plan/union.rs index 43e23850b19e..19d2794caa61 100644 --- a/datafusion/src/physical_plan/union.rs +++ b/datafusion/src/physical_plan/union.rs @@ -218,6 +218,7 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { #[cfg(test)] mod tests { use super::*; + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::{local::LocalFileSystem, ObjectStore}; use crate::test; use crate::{ @@ -236,27 +237,33 @@ mod tests { let (_, files2) = test::create_partitioned_csv("aggregate_test_100.csv", 5)?; let csv = CsvExec::new( - Arc::clone(&fs), - files, - Statistics::default(), - Arc::clone(&schema), + PhysicalPlanConfig { + object_store: Arc::clone(&fs), + file_schema: Arc::clone(&schema), + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); let csv2 = CsvExec::new( - Arc::clone(&fs), - files2, - Statistics::default(), - schema, + PhysicalPlanConfig { + object_store: Arc::clone(&fs), + file_schema: Arc::clone(&schema), + file_groups: files2, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); let union_exec = Arc::new(UnionExec::new(vec![Arc::new(csv), Arc::new(csv2)])); diff --git a/datafusion/src/physical_plan/windows/mod.rs b/datafusion/src/physical_plan/windows/mod.rs index ef420b2c8351..e0fa8b367a3e 100644 --- a/datafusion/src/physical_plan/windows/mod.rs +++ b/datafusion/src/physical_plan/windows/mod.rs @@ -175,6 +175,7 @@ pub(crate) fn find_ranges_in_range<'a>( #[cfg(test)] mod tests { use super::*; + use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::aggregates::AggregateFunction; use crate::physical_plan::expressions::col; @@ -192,15 +193,18 @@ mod tests { let (_, files) = test::create_partitioned_csv("aggregate_test_100.csv", partitions)?; let csv = CsvExec::new( - Arc::new(LocalFileSystem {}), - files, - Statistics::default(), - aggr_test_schema(), + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: aggr_test_schema(), + file_groups: files, + statistics: Statistics::default(), + projection: None, + batch_size: 1024, + limit: None, + table_partition_dims: vec![], + }, true, b',', - None, - 1024, - None, ); let input = Arc::new(csv); From 95216d8c92dc938186df064cf03bd4257acb63ba Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 06/17] [fix] houqp's review --- ballista/rust/core/proto/ballista.proto | 8 ++-- .../core/src/serde/logical_plan/from_proto.rs | 2 +- .../core/src/serde/logical_plan/to_proto.rs | 4 +- .../src/serde/physical_plan/from_proto.rs | 6 +-- .../core/src/serde/physical_plan/to_proto.rs | 6 +-- benchmarks/src/bin/tpch.rs | 2 +- datafusion/src/datasource/file_format/avro.rs | 2 +- datafusion/src/datasource/file_format/csv.rs | 2 +- datafusion/src/datasource/file_format/json.rs | 2 +- datafusion/src/datasource/file_format/mod.rs | 2 +- .../src/datasource/file_format/parquet.rs | 2 +- datafusion/src/datasource/listing/helpers.rs | 48 +++++++++---------- datafusion/src/datasource/listing/table.rs | 16 +++---- datafusion/src/execution/context.rs | 4 +- datafusion/src/execution/options.rs | 4 +- datafusion/src/logical_plan/builder.rs | 2 +- .../src/physical_optimizer/repartition.rs | 4 +- .../src/physical_plan/coalesce_partitions.rs | 2 +- .../src/physical_plan/file_format/avro.rs | 12 ++--- .../src/physical_plan/file_format/csv.rs | 14 +++--- .../physical_plan/file_format/file_stream.rs | 4 +- .../src/physical_plan/file_format/json.rs | 10 ++-- .../src/physical_plan/file_format/parquet.rs | 10 ++-- datafusion/src/physical_plan/filter.rs | 2 +- datafusion/src/physical_plan/limit.rs | 2 +- datafusion/src/physical_plan/projection.rs | 2 +- datafusion/src/physical_plan/sort.rs | 2 +- .../physical_plan/sort_preserving_merge.rs | 4 +- datafusion/src/physical_plan/union.rs | 4 +- datafusion/src/physical_plan/windows/mod.rs | 2 +- 30 files changed, 93 insertions(+), 93 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 7ef00bdb3277..c166b9368739 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -295,7 +295,7 @@ message ListingTableScanNode { ProjectionColumns projection = 4; Schema schema = 5; repeated LogicalExprNode filters = 6; - repeated string table_partition_dims = 7; + repeated string table_partition_cols = 7; bool collect_stat = 8; uint32 target_partitions = 9; oneof FileFormatType { @@ -621,7 +621,7 @@ message ParquetScanExecNode { repeated uint32 projection = 6; ScanLimit limit = 7; Statistics statistics = 8; - repeated string table_partition_dims = 9; + repeated string table_partition_cols = 9; } message CsvScanExecNode { @@ -633,7 +633,7 @@ message CsvScanExecNode { repeated uint32 projection = 6; ScanLimit limit = 7; Statistics statistics = 8; - repeated string table_partition_dims = 9; + repeated string table_partition_cols = 9; } message AvroScanExecNode { @@ -643,7 +643,7 @@ message AvroScanExecNode { repeated uint32 projection = 6; ScanLimit limit = 7; Statistics statistics = 8; - repeated string table_partition_dims = 9; + repeated string table_partition_cols = 9; } enum PartitionMode { diff --git a/ballista/rust/core/src/serde/logical_plan/from_proto.rs b/ballista/rust/core/src/serde/logical_plan/from_proto.rs index 3b25d6fbeaab..259fcb3482a7 100644 --- a/ballista/rust/core/src/serde/logical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/from_proto.rs @@ -191,7 +191,7 @@ impl TryInto for &protobuf::LogicalPlanNode { let options = ListingOptions { file_extension: scan.file_extension.clone(), format: file_format, - table_partition_dims: scan.table_partition_dims.clone(), + table_partition_cols: scan.table_partition_cols.clone(), collect_stat: scan.collect_stat, target_partitions: scan.target_partitions as usize, }; diff --git a/ballista/rust/core/src/serde/logical_plan/to_proto.rs b/ballista/rust/core/src/serde/logical_plan/to_proto.rs index 442fda46ee9a..1d1d48e8a4a3 100644 --- a/ballista/rust/core/src/serde/logical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/logical_plan/to_proto.rs @@ -755,9 +755,9 @@ impl TryInto for &LogicalPlan { .options() .file_extension .clone(), - table_partition_dims: listing_table + table_partition_cols: listing_table .options() - .table_partition_dims + .table_partition_cols .clone(), path: listing_table.table_path().to_owned(), schema: Some(schema), diff --git a/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/ballista/rust/core/src/serde/physical_plan/from_proto.rs index 861be3e29954..8f87fd17e2c1 100644 --- a/ballista/rust/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -137,7 +137,7 @@ impl TryInto> for &protobuf::PhysicalPlanNode { projection: Some(projection), batch_size: scan.batch_size as usize, limit: scan.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_dims: vec![], + table_partition_cols: vec![], }, scan.has_header, str_to_byte(&scan.delimiter)?, @@ -161,7 +161,7 @@ impl TryInto> for &protobuf::PhysicalPlanNode { projection: Some(projection), batch_size: scan.batch_size as usize, limit: scan.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_dims: scan.table_partition_dims.clone(), + table_partition_cols: scan.table_partition_cols.clone(), }, // TODO predicate should be de-serialized None, @@ -184,7 +184,7 @@ impl TryInto> for &protobuf::PhysicalPlanNode { projection: Some(projection), batch_size: scan.batch_size as usize, limit: scan.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_dims: vec![], + table_partition_cols: vec![], }))) } PhysicalPlanType::CoalesceBatches(coalesce_batches) => { diff --git a/ballista/rust/core/src/serde/physical_plan/to_proto.rs b/ballista/rust/core/src/serde/physical_plan/to_proto.rs index 3c061a029b7b..51f0c4126c8d 100644 --- a/ballista/rust/core/src/serde/physical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/to_proto.rs @@ -272,7 +272,7 @@ impl TryInto for Arc { has_header: exec.has_header(), delimiter: byte_to_string(exec.delimiter())?, batch_size: exec.batch_size() as u32, - table_partition_dims: exec.table_partition_dims().to_vec(), + table_partition_cols: exec.table_partition_cols().to_vec(), }, )), }) @@ -299,7 +299,7 @@ impl TryInto for Arc { .map(|n| *n as u32) .collect(), batch_size: exec.batch_size() as u32, - table_partition_dims: exec.table_partition_dims().to_vec(), + table_partition_cols: exec.table_partition_cols().to_vec(), }, )), }) @@ -330,7 +330,7 @@ impl TryInto for Arc { .collect(), schema: Some(exec.file_schema().as_ref().into()), batch_size: exec.batch_size() as u32, - table_partition_dims: exec.table_partition_dims().to_vec(), + table_partition_cols: exec.table_partition_cols().to_vec(), }, )), }) diff --git a/benchmarks/src/bin/tpch.rs b/benchmarks/src/bin/tpch.rs index 8b4fd3e41cac..7bc6510ac2ed 100644 --- a/benchmarks/src/bin/tpch.rs +++ b/benchmarks/src/bin/tpch.rs @@ -496,7 +496,7 @@ fn get_table( file_extension: extension.to_owned(), target_partitions, collect_stat: true, - table_partition_dims: vec![], + table_partition_cols: vec![], }; Ok(Arc::new(ListingTable::new( diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index 317658f5aa69..a807022eacc7 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -359,7 +359,7 @@ mod tests { projection: projection.clone(), batch_size, limit, - table_partition_dims: vec![], + table_partition_cols: vec![], }, &[], ) diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index 885acf23de5a..0574ad8de1d9 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -271,7 +271,7 @@ mod tests { projection: projection.clone(), batch_size, limit, - table_partition_dims: vec![], + table_partition_cols: vec![], }, &[], ) diff --git a/datafusion/src/datasource/file_format/json.rs b/datafusion/src/datasource/file_format/json.rs index 372c5b99c154..72bbee665a61 100644 --- a/datafusion/src/datasource/file_format/json.rs +++ b/datafusion/src/datasource/file_format/json.rs @@ -224,7 +224,7 @@ mod tests { projection: projection.clone(), batch_size, limit, - table_partition_dims: vec![], + table_partition_cols: vec![], }, &[], ) diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index 83d30b5e938a..a373bd046cdb 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -54,7 +54,7 @@ pub struct PhysicalPlanConfig { /// The minimum number of records required from this source plan pub limit: Option, /// The partitioning column names - pub table_partition_dims: Vec, + pub table_partition_cols: Vec, } /// This trait abstracts all the file format specific implementations diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs index c1d9cb45a67e..819f37448636 100644 --- a/datafusion/src/datasource/file_format/parquet.rs +++ b/datafusion/src/datasource/file_format/parquet.rs @@ -604,7 +604,7 @@ mod tests { projection: projection.clone(), batch_size, limit, - table_partition_dims: vec![], + table_partition_cols: vec![], }, &[], ) diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs index 17c6a44bafa2..cec9d6762d70 100644 --- a/datafusion/src/datasource/listing/helpers.rs +++ b/datafusion/src/datasource/listing/helpers.rs @@ -59,10 +59,7 @@ pub fn split_files( if partitioned_files.is_empty() { return vec![]; } - let mut chunk_size = partitioned_files.len() / n; - if partitioned_files.len() % n > 0 { - chunk_size += 1; - } + let chunk_size = (partitioned_files.len() + n - 1) / n; partitioned_files .chunks(chunk_size) .map(|c| c.to_vec()) @@ -70,7 +67,7 @@ pub fn split_files( } /// Discover the partitions on the given path and prune out files -/// relative to irrelevant partitions using `filters` expressions +/// that belong to irrelevant partitions using `filters` expressions. /// TODO for tables with many files (10k+), it will usually more efficient /// to first list the folders relative to the first partition dimension, /// prune those, then list only the contain of the remaining folders. @@ -79,9 +76,9 @@ pub async fn pruned_partition_list( table_path: &str, filters: &[Expr], file_extension: &str, - table_partition_dims: &[String], + table_partition_cols: &[String], ) -> Result { - if table_partition_dims.is_empty() { + if table_partition_cols.is_empty() || filters.is_empty() { Ok(Box::pin( store .list_file_with_suffix(table_path, file_extension) @@ -96,18 +93,22 @@ pub async fn pruned_partition_list( } else { let applicable_filters = filters .iter() - .filter(|f| expr_applicable_for_cols(table_partition_dims, f)); + .filter(|f| expr_applicable_for_cols(table_partition_cols, f)); - let table_partition_dims = table_partition_dims.to_vec(); + let table_partition_cols = table_partition_cols.to_vec(); let stream_path = table_path.to_owned(); // TODO avoid collecting but have a streaming memory table instead let batches: Vec = store .list_file_with_suffix(table_path, file_extension) .await? - .chunks(64) + // TODO we set an arbitrary high batch size here, it does not matter as we list + // all the files anyway. This number will need to be adjusted according to the object + // store if we switch to a streaming-stlye pruning of the files. For instance S3 lists + // 1000 items at a time so batches of 1000 would be ideal with S3 as store. + .chunks(1024) .map(|v| v.into_iter().collect::>>()) .map(move |metas| { - paths_to_batch(&table_partition_dims, &stream_path, &metas?) + paths_to_batch(&table_partition_cols, &stream_path, &metas?) }) .try_collect() .await?; @@ -137,20 +138,20 @@ pub async fn pruned_partition_list( /// - ... one column by partition ... /// Note: For the last modified date, this looses precisions higher than millisecond. fn paths_to_batch( - table_partition_dims: &[String], + table_partition_cols: &[String], table_path: &str, metas: &[FileMeta], ) -> Result { let mut key_builder = StringBuilder::new(metas.len()); let mut length_builder = UInt64Builder::new(metas.len()); let mut modified_builder = Date64Builder::new(metas.len()); - let mut partition_builders = table_partition_dims + let mut partition_builders = table_partition_cols .iter() .map(|_| StringBuilder::new(metas.len())) .collect::>(); for file_meta in metas { if let Some(partition_values) = - parse_partitions_for_path(table_path, file_meta.path(), table_partition_dims) + parse_partitions_for_path(table_path, file_meta.path(), table_partition_cols) { key_builder.append_value(file_meta.path())?; length_builder.append_value(file_meta.size())?; @@ -178,11 +179,11 @@ fn paths_to_batch( // put the schema together let mut fields = vec![ - Field::new(FILE_SIZE_COLUMN_NAME, DataType::Utf8, false), - Field::new(FILE_PATH_COLUMN_NAME, DataType::UInt64, false), + Field::new(FILE_PATH_COLUMN_NAME, DataType::Utf8, false), + Field::new(FILE_SIZE_COLUMN_NAME, DataType::UInt64, false), Field::new(FILE_MODIFIED_COLUMN_NAME, DataType::Date64, false), ]; - for pn in table_partition_dims { + for pn in table_partition_cols { fields.push(Field::new(pn, DataType::Utf8, false)); } @@ -233,11 +234,11 @@ fn batches_to_paths(batches: &[RecordBatch]) -> Vec { } /// Extract the partition values for the given `file_path` (in the given `table_path`) -/// associated to the partitions defined by `table_partition_dims` +/// associated to the partitions defined by `table_partition_cols` fn parse_partitions_for_path<'a>( table_path: &str, file_path: &'a str, - table_partition_dims: &[String], + table_partition_cols: &[String], ) -> Option> { let subpath = file_path.strip_prefix(table_path)?; @@ -248,11 +249,10 @@ fn parse_partitions_for_path<'a>( }; let mut part_values = vec![]; - for (path, pn) in subpath.split('/').zip(table_partition_dims) { - if let Some(val) = path.strip_prefix(&format!("{}=", pn)) { - part_values.push(val); - } else { - return None; + for (path, pn) in subpath.split('/').zip(table_partition_cols) { + match path.split_once('=') { + Some((name, val)) if name == pn => part_values.push(val), + _ => return None, } } Some(part_values) diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index 8e96d5e54baa..1efdd03f20b8 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -52,7 +52,7 @@ pub struct ListingOptions { /// - If there is a third level of partitioning it will be ignored. /// - Files that don't follow this partitioning will be ignored. /// Note that only `DataType::Utf8` is supported for the column type. - pub table_partition_dims: Vec, + pub table_partition_cols: Vec, /// 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. @@ -73,7 +73,7 @@ impl ListingOptions { Self { file_extension: String::new(), format, - table_partition_dims: vec![], + table_partition_cols: vec![], collect_stat: false, target_partitions: 1, } @@ -97,7 +97,7 @@ impl ListingOptions { let file_schema = self.format.infer_schema(Box::pin(file_stream)).await?; // Add the partition columns to the file schema let mut fields = file_schema.fields().clone(); - for part in &self.table_partition_dims { + for part in &self.table_partition_cols { fields.push(Field::new(part, DataType::Utf8, false)); } Ok(Arc::new(Schema::new(fields))) @@ -189,7 +189,7 @@ impl TableProvider for ListingTable { projection: projection.clone(), batch_size, limit, - table_partition_dims: self.options.table_partition_dims.clone(), + table_partition_cols: self.options.table_partition_cols.clone(), }, filters, ) @@ -218,7 +218,7 @@ impl ListingTable { &self.table_path, filters, &self.options.file_extension, - &self.options.table_partition_dims, + &self.options.table_partition_cols, ) .await?; @@ -288,7 +288,7 @@ mod tests { let opt = ListingOptions { file_extension: ".avro".to_owned(), format: Arc::new(AvroFormat {}), - table_partition_dims: vec![String::from("p1")], + table_partition_cols: vec![String::from("p1")], target_partitions: 4, collect_stat: true, }; @@ -380,7 +380,7 @@ mod tests { let opt = ListingOptions { file_extension: "parquet".to_owned(), format: Arc::new(ParquetFormat::default()), - table_partition_dims: vec![], + table_partition_cols: vec![], target_partitions: 2, collect_stat: true, }; @@ -410,7 +410,7 @@ mod tests { let opt = ListingOptions { file_extension: "".to_owned(), format: Arc::new(format), - table_partition_dims: vec![], + table_partition_cols: vec![], target_partitions, collect_stat: true, }; diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index 2dca98747b00..152d02f7308d 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -221,7 +221,7 @@ impl ExecutionContext { .unwrap() .config .target_partitions, - table_partition_dims: vec![], + table_partition_cols: vec![], }; // TODO make schema in CreateExternalTable optional instead of empty @@ -442,7 +442,7 @@ impl ExecutionContext { collect_stat: true, file_extension: DEFAULT_PARQUET_EXTENSION.to_owned(), target_partitions, - table_partition_dims: vec![], + table_partition_cols: vec![], }; self.register_listing_table(name, uri, listing_options, None) diff --git a/datafusion/src/execution/options.rs b/datafusion/src/execution/options.rs index 255cf9285240..c6b5ff646ea3 100644 --- a/datafusion/src/execution/options.rs +++ b/datafusion/src/execution/options.rs @@ -108,7 +108,7 @@ impl<'a> CsvReadOptions<'a> { collect_stat: false, file_extension: self.file_extension.to_owned(), target_partitions, - table_partition_dims: vec![], + table_partition_cols: vec![], } } } @@ -143,7 +143,7 @@ impl<'a> AvroReadOptions<'a> { collect_stat: false, file_extension: self.file_extension.to_owned(), target_partitions, - table_partition_dims: vec![], + table_partition_cols: vec![], } } } diff --git a/datafusion/src/logical_plan/builder.rs b/datafusion/src/logical_plan/builder.rs index 6865ec0da287..693bf78fbe0e 100644 --- a/datafusion/src/logical_plan/builder.rs +++ b/datafusion/src/logical_plan/builder.rs @@ -277,7 +277,7 @@ impl LogicalPlanBuilder { collect_stat: true, file_extension: DEFAULT_PARQUET_EXTENSION.to_owned(), target_partitions, - table_partition_dims: vec![], + table_partition_cols: vec![], }; let path: String = path.into(); diff --git a/datafusion/src/physical_optimizer/repartition.rs b/datafusion/src/physical_optimizer/repartition.rs index afd4af5c7f22..28b309589149 100644 --- a/datafusion/src/physical_optimizer/repartition.rs +++ b/datafusion/src/physical_optimizer/repartition.rs @@ -130,7 +130,7 @@ mod tests { projection: None, batch_size: 2048, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, None, )), @@ -172,7 +172,7 @@ mod tests { projection: None, batch_size: 2048, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, None, )), diff --git a/datafusion/src/physical_plan/coalesce_partitions.rs b/datafusion/src/physical_plan/coalesce_partitions.rs index 94198aec2afe..285daeac29e4 100644 --- a/datafusion/src/physical_plan/coalesce_partitions.rs +++ b/datafusion/src/physical_plan/coalesce_partitions.rs @@ -229,7 +229,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index 076446bd046d..301b0c3a4dbd 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -47,7 +47,7 @@ pub struct AvroExec { projected_schema: SchemaRef, batch_size: usize, limit: Option, - table_partition_dims: Vec, + table_partition_cols: Vec, } impl AvroExec { @@ -68,7 +68,7 @@ impl AvroExec { projected_schema, batch_size: base_config.batch_size, limit: base_config.limit, - table_partition_dims: base_config.table_partition_dims, + table_partition_cols: base_config.table_partition_cols, } } /// List of data files @@ -92,8 +92,8 @@ impl AvroExec { self.limit } /// Partitioning column names - pub fn table_partition_dims(&self) -> &[String] { - &self.table_partition_dims + pub fn table_partition_cols(&self) -> &[String] { + &self.table_partition_cols } } @@ -170,7 +170,7 @@ impl ExecutionPlan for AvroExec { fun, Arc::clone(&self.projected_schema), self.limit, - self.table_partition_dims.clone(), + self.table_partition_cols.clone(), ))) } @@ -225,7 +225,7 @@ mod tests { projection: Some(vec![0, 1, 2]), batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }); assert_eq!(avro_exec.output_partitioning().partition_count(), 1); diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index 77590d6fae5c..c50866b8e095 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -49,7 +49,7 @@ pub struct CsvExec { projection: Option>, batch_size: usize, limit: Option, - table_partition_dims: Vec, + table_partition_cols: Vec, } impl CsvExec { @@ -72,7 +72,7 @@ impl CsvExec { projected_schema, batch_size: base_config.batch_size, limit: base_config.limit, - table_partition_dims: base_config.table_partition_dims, + table_partition_cols: base_config.table_partition_cols, } } @@ -105,8 +105,8 @@ impl CsvExec { self.limit } /// Partitioning column names - pub fn table_partition_dims(&self) -> &[String] { - &self.table_partition_dims + pub fn table_partition_cols(&self) -> &[String] { + &self.table_partition_cols } } @@ -173,7 +173,7 @@ impl ExecutionPlan for CsvExec { fun, Arc::clone(&self.projected_schema), self.limit, - self.table_partition_dims.clone(), + self.table_partition_cols.clone(), ))) } @@ -225,7 +225,7 @@ mod tests { projection: Some(vec![0, 2, 4]), batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', @@ -259,7 +259,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', diff --git a/datafusion/src/physical_plan/file_format/file_stream.rs b/datafusion/src/physical_plan/file_format/file_stream.rs index 5863a136ea8b..5c2c4525b0a0 100644 --- a/datafusion/src/physical_plan/file_format/file_stream.rs +++ b/datafusion/src/physical_plan/file_format/file_stream.rs @@ -97,10 +97,10 @@ impl FileStream { file_reader: F, schema: SchemaRef, limit: Option, - table_partition_dims: Vec, + table_partition_cols: Vec, ) -> Self { let mut projected_partition_indexes = HashMap::new(); - for (partition_idx, partition_name) in table_partition_dims.iter().enumerate() { + for (partition_idx, partition_name) in table_partition_cols.iter().enumerate() { if let Ok(schema_idx) = schema.index_of(partition_name) { projected_partition_indexes.insert(partition_idx, schema_idx); } diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index a705e9d70f7d..d1309fab938d 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -42,7 +42,7 @@ pub struct NdJsonExec { projected_schema: SchemaRef, batch_size: usize, limit: Option, - table_partition_dims: Vec, + table_partition_cols: Vec, } impl NdJsonExec { @@ -63,7 +63,7 @@ impl NdJsonExec { projected_schema, batch_size: base_config.batch_size, limit: base_config.limit, - table_partition_dims: base_config.table_partition_dims, + table_partition_cols: base_config.table_partition_cols, } } } @@ -127,7 +127,7 @@ impl ExecutionPlan for NdJsonExec { fun, Arc::clone(&self.projected_schema), self.limit, - self.table_partition_dims.clone(), + self.table_partition_cols.clone(), ))) } @@ -187,7 +187,7 @@ mod tests { projection: None, batch_size: 1024, limit: Some(3), - table_partition_dims: vec![], + table_partition_cols: vec![], }); // TODO: this is not where schema inference should be tested @@ -241,7 +241,7 @@ mod tests { projection: Some(vec![0, 2]), batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 2); diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index 550af996f1d7..8cdab90e14e8 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -83,7 +83,7 @@ pub struct ParquetExec { /// Optional limit of the number of rows limit: Option, /// Partioning column names - table_partition_dims: Vec, + table_partition_cols: Vec, } /// Stores metrics about the parquet execution for a particular parquet file @@ -144,7 +144,7 @@ impl ParquetExec { batch_size: base_config.batch_size, statistics: projected_statistics, limit: base_config.limit, - table_partition_dims: base_config.table_partition_dims, + table_partition_cols: base_config.table_partition_cols, } } @@ -165,8 +165,8 @@ impl ParquetExec { self.limit } /// Partitioning column names - pub fn table_partition_dims(&self) -> &[String] { - &self.table_partition_dims + pub fn table_partition_cols(&self) -> &[String] { + &self.table_partition_cols } } @@ -521,7 +521,7 @@ mod tests { projection: Some(vec![0, 1, 2]), batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, None, ); diff --git a/datafusion/src/physical_plan/filter.rs b/datafusion/src/physical_plan/filter.rs index 1b99b47ed458..bab9f1e974d8 100644 --- a/datafusion/src/physical_plan/filter.rs +++ b/datafusion/src/physical_plan/filter.rs @@ -249,7 +249,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', diff --git a/datafusion/src/physical_plan/limit.rs b/datafusion/src/physical_plan/limit.rs index 9fa9270f18d0..62df87987f50 100644 --- a/datafusion/src/physical_plan/limit.rs +++ b/datafusion/src/physical_plan/limit.rs @@ -408,7 +408,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', diff --git a/datafusion/src/physical_plan/projection.rs b/datafusion/src/physical_plan/projection.rs index fc3bfd21ad76..4cc53196eb6a 100644 --- a/datafusion/src/physical_plan/projection.rs +++ b/datafusion/src/physical_plan/projection.rs @@ -284,7 +284,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', diff --git a/datafusion/src/physical_plan/sort.rs b/datafusion/src/physical_plan/sort.rs index 59319e9a1b1f..fffebd6008d6 100644 --- a/datafusion/src/physical_plan/sort.rs +++ b/datafusion/src/physical_plan/sort.rs @@ -339,7 +339,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', diff --git a/datafusion/src/physical_plan/sort_preserving_merge.rs b/datafusion/src/physical_plan/sort_preserving_merge.rs index 7e66c5ad7ae4..ee0285c37780 100644 --- a/datafusion/src/physical_plan/sort_preserving_merge.rs +++ b/datafusion/src/physical_plan/sort_preserving_merge.rs @@ -945,7 +945,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', @@ -1028,7 +1028,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', diff --git a/datafusion/src/physical_plan/union.rs b/datafusion/src/physical_plan/union.rs index 19d2794caa61..01cb457351a3 100644 --- a/datafusion/src/physical_plan/union.rs +++ b/datafusion/src/physical_plan/union.rs @@ -245,7 +245,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', @@ -260,7 +260,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', diff --git a/datafusion/src/physical_plan/windows/mod.rs b/datafusion/src/physical_plan/windows/mod.rs index e0fa8b367a3e..6ff4a49fa29d 100644 --- a/datafusion/src/physical_plan/windows/mod.rs +++ b/datafusion/src/physical_plan/windows/mod.rs @@ -201,7 +201,7 @@ mod tests { projection: None, batch_size: 1024, limit: None, - table_partition_dims: vec![], + table_partition_cols: vec![], }, true, b',', From 26bf10524cfa3859a0024b8441614784029312d4 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 07/17] [fix] ListingTable takes schema without partition cols This helps avoid providing schemas with wrong datatypes. --- datafusion/src/datasource/file_format/mod.rs | 3 ++- datafusion/src/datasource/listing/table.rs | 23 +++++++++++-------- .../src/physical_plan/file_format/csv.rs | 2 -- 3 files changed, 15 insertions(+), 13 deletions(-) diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index a373bd046cdb..e711a0b9744b 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -41,7 +41,8 @@ use super::PartitionedFile; pub struct PhysicalPlanConfig { /// Store from which the `files` should be fetched pub object_store: Arc, - /// Schema before projection + /// Schema before projection. It contains the columns that are expected + /// to be in the file followed by the table partition columns. pub file_schema: SchemaRef, /// List of files to be processed, grouped into partitions pub file_groups: Vec>, diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index 1efdd03f20b8..2361c24ae63e 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -80,7 +80,7 @@ impl ListingOptions { } /// Infer the schema of the files at the given path on the provided object store. - /// The inferred schema should include the partitioning columns. + /// 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 @@ -95,12 +95,7 @@ impl ListingOptions { .await? .map(move |file_meta| object_store.file_reader(file_meta?.sized_file)); let file_schema = self.format.infer_schema(Box::pin(file_stream)).await?; - // Add the partition columns to the file schema - let mut fields = file_schema.fields().clone(); - for part in &self.table_partition_cols { - fields.push(Field::new(part, DataType::Utf8, false)); - } - Ok(Arc::new(Schema::new(fields))) + Ok(file_schema) } } @@ -109,24 +104,32 @@ impl ListingOptions { pub struct ListingTable { object_store: Arc, table_path: String, - /// the schema also contains the partition columns + /// File fields + partition columns schema: SchemaRef, options: ListingOptions, } impl ListingTable { /// Create new table that lists the FS to get the files to scan. + /// The provided `schema` must be resolved before creating the table + /// and should contain the fields of the file without the table + /// partitioning columns. pub fn new( object_store: Arc, table_path: String, - // the schema must be resolved before creating the table schema: SchemaRef, options: ListingOptions, ) -> Self { + // Add the partition columns to the file schema + let mut fields = schema.fields().clone(); + for part in &options.table_partition_cols { + fields.push(Field::new(part, DataType::Utf8, false)); + } + Self { object_store, table_path, - schema, + schema: Arc::new(Schema::new(fields)), options, } } diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index c50866b8e095..a96d8f314f24 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -39,9 +39,7 @@ use super::file_stream::{BatchIter, FileStream}; pub struct CsvExec { object_store: Arc, file_groups: Vec>, - /// Schema representing the CSV file file_schema: SchemaRef, - /// Schema after the projection has been applied projected_schema: SchemaRef, statistics: Statistics, has_header: bool, From c34193de934deb0be8b410907cca6223ac4524da Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 08/17] [feat] added partition columns for parquet refactored partition column projection and added test --- ballista/rust/core/proto/ballista.proto | 36 +- .../src/serde/physical_plan/from_proto.rs | 110 ++--- .../core/src/serde/physical_plan/to_proto.rs | 105 ++--- datafusion/src/datasource/file_format/avro.rs | 4 +- datafusion/src/datasource/file_format/csv.rs | 4 +- datafusion/src/datasource/file_format/mod.rs | 26 +- datafusion/src/datasource/listing/helpers.rs | 1 + datafusion/src/datasource/listing/table.rs | 41 +- .../src/physical_optimizer/repartition.rs | 3 +- .../src/physical_plan/coalesce_partitions.rs | 3 +- .../src/physical_plan/file_format/avro.rs | 179 ++++---- .../src/physical_plan/file_format/csv.rs | 191 +++++---- .../physical_plan/file_format/file_stream.rs | 89 +--- .../src/physical_plan/file_format/json.rs | 61 +-- .../src/physical_plan/file_format/mod.rs | 398 ++++++++++++++++-- .../src/physical_plan/file_format/parquet.rs | 164 +++++--- datafusion/src/physical_plan/filter.rs | 3 +- datafusion/src/physical_plan/limit.rs | 3 +- datafusion/src/physical_plan/projection.rs | 3 +- datafusion/src/physical_plan/sort.rs | 6 +- .../physical_plan/sort_preserving_merge.rs | 3 +- datafusion/src/physical_plan/union.rs | 7 +- datafusion/src/physical_plan/windows/mod.rs | 3 +- 23 files changed, 855 insertions(+), 588 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index c166b9368739..62b3185314fd 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -614,36 +614,28 @@ message ScanLimit { uint32 limit = 1; } -message ParquetScanExecNode { +message FileScanExecConf { repeated FileGroup file_groups = 1; Schema schema = 2; - uint32 batch_size = 4; - repeated uint32 projection = 6; - ScanLimit limit = 7; - Statistics statistics = 8; - repeated string table_partition_cols = 9; + uint32 batch_size = 3; + repeated uint32 projection = 4; + ScanLimit limit = 5; + Statistics statistics = 6; + repeated string table_partition_cols = 7; +} + +message ParquetScanExecNode { + FileScanExecConf base_conf = 1; } message CsvScanExecNode { - repeated FileGroup file_groups = 1; - Schema schema = 2; - bool has_header = 3; - uint32 batch_size = 4; - string delimiter = 5; - repeated uint32 projection = 6; - ScanLimit limit = 7; - Statistics statistics = 8; - repeated string table_partition_cols = 9; + FileScanExecConf base_conf = 1; + bool has_header = 2; + string delimiter = 3; } message AvroScanExecNode { - repeated FileGroup file_groups = 1; - Schema schema = 2; - uint32 batch_size = 4; - repeated uint32 projection = 6; - ScanLimit limit = 7; - Statistics statistics = 8; - repeated string table_partition_cols = 9; + FileScanExecConf base_conf = 1; } enum PartitionMode { diff --git a/ballista/rust/core/src/serde/physical_plan/from_proto.rs b/ballista/rust/core/src/serde/physical_plan/from_proto.rs index 8f87fd17e2c1..99d2de03258f 100644 --- a/ballista/rust/core/src/serde/physical_plan/from_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/from_proto.rs @@ -35,7 +35,6 @@ use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::catalog::catalog::{ CatalogList, CatalogProvider, MemoryCatalogList, MemoryCatalogProvider, }; -use datafusion::datasource::file_format::PhysicalPlanConfig; use datafusion::datasource::object_store::local::LocalFileSystem; use datafusion::datasource::object_store::{FileMeta, ObjectStoreRegistry, SizedFile}; use datafusion::datasource::PartitionedFile; @@ -47,7 +46,9 @@ use datafusion::logical_plan::{ }; use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateFunction}; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion::physical_plan::file_format::{AvroExec, CsvExec, ParquetExec}; +use datafusion::physical_plan::file_format::{ + AvroExec, CsvExec, ParquetExec, PhysicalPlanConfig, +}; use datafusion::physical_plan::hash_aggregate::{AggregateMode, HashAggregateExec}; use datafusion::physical_plan::hash_join::PartitionMode; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; @@ -119,74 +120,21 @@ impl TryInto> for &protobuf::PhysicalPlanNode { .try_into()?; Ok(Arc::new(FilterExec::try_new(predicate, input)?)) } - PhysicalPlanType::CsvScan(scan) => { - let schema = Arc::new(convert_required!(scan.schema)?); - let projection = scan.projection.iter().map(|i| *i as usize).collect(); - let statistics = convert_required!(scan.statistics)?; - - Ok(Arc::new(CsvExec::new( - PhysicalPlanConfig { - object_store: Arc::new(LocalFileSystem {}), - file_schema: schema, - file_groups: scan - .file_groups - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?, - statistics, - projection: Some(projection), - batch_size: scan.batch_size as usize, - limit: scan.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_cols: vec![], - }, - scan.has_header, - str_to_byte(&scan.delimiter)?, - ))) - } + PhysicalPlanType::CsvScan(scan) => Ok(Arc::new(CsvExec::new( + scan.base_conf.as_ref().unwrap().try_into()?, + scan.has_header, + str_to_byte(&scan.delimiter)?, + ))), PhysicalPlanType::ParquetScan(scan) => { - let schema = Arc::new(convert_required!(scan.schema)?); - let projection = scan.projection.iter().map(|i| *i as usize).collect(); - let statistics = convert_required!(scan.statistics)?; - Ok(Arc::new(ParquetExec::new( - PhysicalPlanConfig { - object_store: Arc::new(LocalFileSystem {}), - file_schema: schema, - file_groups: scan - .file_groups - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?, - statistics, - projection: Some(projection), - batch_size: scan.batch_size as usize, - limit: scan.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_cols: scan.table_partition_cols.clone(), - }, + scan.base_conf.as_ref().unwrap().try_into()?, // TODO predicate should be de-serialized None, ))) } - PhysicalPlanType::AvroScan(scan) => { - let schema = Arc::new(convert_required!(scan.schema)?); - let projection = scan.projection.iter().map(|i| *i as usize).collect(); - let statistics = convert_required!(scan.statistics)?; - - Ok(Arc::new(AvroExec::new(PhysicalPlanConfig { - object_store: Arc::new(LocalFileSystem {}), - file_schema: schema, - file_groups: scan - .file_groups - .iter() - .map(|f| f.try_into()) - .collect::, _>>()?, - statistics, - projection: Some(projection), - batch_size: scan.batch_size as usize, - limit: scan.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_cols: vec![], - }))) - } + PhysicalPlanType::AvroScan(scan) => Ok(Arc::new(AvroExec::new( + scan.base_conf.as_ref().unwrap().try_into()?, + ))), PhysicalPlanType::CoalesceBatches(coalesce_batches) => { let input: Arc = convert_box_required!(coalesce_batches.input)?; @@ -818,3 +766,37 @@ impl TryInto for &protobuf::Statistics { }) } } + +impl TryInto for &protobuf::FileScanExecConf { + type Error = BallistaError; + + fn try_into(self) -> Result { + let schema = Arc::new(convert_required!(self.schema)?); + let projection = self + .projection + .iter() + .map(|i| *i as usize) + .collect::>(); + let projection = if projection.is_empty() { + None + } else { + Some(projection) + }; + let statistics = convert_required!(self.statistics)?; + + Ok(PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_schema: schema, + file_groups: self + .file_groups + .iter() + .map(|f| f.try_into()) + .collect::, _>>()?, + statistics, + projection, + batch_size: self.batch_size as usize, + limit: self.limit.as_ref().map(|sl| sl.limit as usize), + table_partition_cols: vec![], + }) + } +} diff --git a/ballista/rust/core/src/serde/physical_plan/to_proto.rs b/ballista/rust/core/src/serde/physical_plan/to_proto.rs index 51f0c4126c8d..afbb02a4f216 100644 --- a/ballista/rust/core/src/serde/physical_plan/to_proto.rs +++ b/ballista/rust/core/src/serde/physical_plan/to_proto.rs @@ -26,7 +26,6 @@ use std::{ sync::Arc, }; -use datafusion::physical_plan::hash_aggregate::AggregateMode; use datafusion::physical_plan::hash_join::{HashJoinExec, PartitionMode}; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion::physical_plan::projection::ProjectionExec; @@ -43,6 +42,9 @@ use datafusion::physical_plan::{ file_format::ParquetExec, }; use datafusion::physical_plan::{file_format::AvroExec, filter::FilterExec}; +use datafusion::physical_plan::{ + file_format::PhysicalPlanConfig, hash_aggregate::AggregateMode, +}; use datafusion::{ datasource::PartitionedFile, physical_plan::coalesce_batches::CoalesceBatchesExec, }; @@ -244,93 +246,29 @@ impl TryInto for Arc { ))), }) } else if let Some(exec) = plan.downcast_ref::() { - let file_groups = exec - .file_groups() - .iter() - .map(|p| p.as_slice().try_into()) - .collect::, _>>()?; Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::CsvScan( protobuf::CsvScanExecNode { - file_groups, - statistics: Some((&exec.statistics()).into()), - limit: exec - .limit() - .map(|l| protobuf::ScanLimit { limit: l as u32 }), - projection: exec - .projection() - .as_ref() - .ok_or_else(|| { - BallistaError::General( - "projection in CsvExec does not exist.".to_owned(), - ) - })? - .iter() - .map(|n| *n as u32) - .collect(), - schema: Some(exec.file_schema().as_ref().into()), + base_conf: Some(exec.base_config().try_into()?), has_header: exec.has_header(), delimiter: byte_to_string(exec.delimiter())?, - batch_size: exec.batch_size() as u32, - table_partition_cols: exec.table_partition_cols().to_vec(), }, )), }) } else if let Some(exec) = plan.downcast_ref::() { - let file_groups = exec - .file_groups() - .iter() - .map(|p| p.as_slice().try_into()) - .collect::, _>>()?; - Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::ParquetScan( protobuf::ParquetScanExecNode { - file_groups, - statistics: Some((&exec.statistics()).into()), - limit: exec - .limit() - .map(|l| protobuf::ScanLimit { limit: l as u32 }), - schema: Some(exec.schema().as_ref().into()), - projection: exec - .projection() - .as_ref() - .iter() - .map(|n| *n as u32) - .collect(), - batch_size: exec.batch_size() as u32, - table_partition_cols: exec.table_partition_cols().to_vec(), + base_conf: Some(exec.base_config().try_into()?), + // TODO serialize predicates }, )), }) } else if let Some(exec) = plan.downcast_ref::() { - let file_groups = exec - .file_groups() - .iter() - .map(|p| p.as_slice().try_into()) - .collect::, _>>()?; Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::AvroScan( protobuf::AvroScanExecNode { - file_groups, - statistics: Some((&exec.statistics()).into()), - limit: exec - .limit() - .map(|l| protobuf::ScanLimit { limit: l as u32 }), - projection: exec - .projection() - .as_ref() - .ok_or_else(|| { - BallistaError::General( - "projection in AvroExec does not exist.".to_owned(), - ) - })? - .iter() - .map(|n| *n as u32) - .collect(), - schema: Some(exec.file_schema().as_ref().into()), - batch_size: exec.batch_size() as u32, - table_partition_cols: exec.table_partition_cols().to_vec(), + base_conf: Some(exec.base_config().try_into()?), }, )), }) @@ -737,3 +675,32 @@ impl From<&Statistics> for protobuf::Statistics { } } } + +impl TryFrom<&PhysicalPlanConfig> for protobuf::FileScanExecConf { + type Error = BallistaError; + fn try_from( + conf: &PhysicalPlanConfig, + ) -> Result { + let file_groups = conf + .file_groups + .iter() + .map(|p| p.as_slice().try_into()) + .collect::, _>>()?; + + Ok(protobuf::FileScanExecConf { + file_groups, + statistics: Some((&conf.statistics).into()), + limit: conf.limit.map(|l| protobuf::ScanLimit { limit: l as u32 }), + projection: conf + .projection + .as_ref() + .unwrap_or(&vec![]) + .iter() + .map(|n| *n as u32) + .collect(), + schema: Some(conf.file_schema.as_ref().into()), + batch_size: conf.batch_size as u32, + table_partition_cols: conf.table_partition_cols.to_vec(), + }) + } +} diff --git a/datafusion/src/datasource/file_format/avro.rs b/datafusion/src/datasource/file_format/avro.rs index a807022eacc7..515584b16c03 100644 --- a/datafusion/src/datasource/file_format/avro.rs +++ b/datafusion/src/datasource/file_format/avro.rs @@ -25,12 +25,12 @@ use arrow::{self, datatypes::SchemaRef}; use async_trait::async_trait; use futures::StreamExt; -use super::{FileFormat, PhysicalPlanConfig}; +use super::FileFormat; use crate::avro_to_arrow::read_avro_schema_from_reader; use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::Result; use crate::logical_plan::Expr; -use crate::physical_plan::file_format::AvroExec; +use crate::physical_plan::file_format::{AvroExec, PhysicalPlanConfig}; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; diff --git a/datafusion/src/datasource/file_format/csv.rs b/datafusion/src/datasource/file_format/csv.rs index 0574ad8de1d9..337511316c51 100644 --- a/datafusion/src/datasource/file_format/csv.rs +++ b/datafusion/src/datasource/file_format/csv.rs @@ -25,11 +25,11 @@ use arrow::{self, datatypes::SchemaRef}; use async_trait::async_trait; use futures::StreamExt; -use super::{FileFormat, PhysicalPlanConfig}; +use super::FileFormat; use crate::datasource::object_store::{ObjectReader, ObjectReaderStream}; use crate::error::Result; use crate::logical_plan::Expr; -use crate::physical_plan::file_format::CsvExec; +use crate::physical_plan::file_format::{CsvExec, PhysicalPlanConfig}; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; diff --git a/datafusion/src/datasource/file_format/mod.rs b/datafusion/src/datasource/file_format/mod.rs index e711a0b9744b..54491615fc4c 100644 --- a/datafusion/src/datasource/file_format/mod.rs +++ b/datafusion/src/datasource/file_format/mod.rs @@ -29,34 +29,12 @@ use std::sync::Arc; use crate::arrow::datatypes::SchemaRef; use crate::error::Result; use crate::logical_plan::Expr; +use crate::physical_plan::file_format::PhysicalPlanConfig; use crate::physical_plan::{ExecutionPlan, Statistics}; use async_trait::async_trait; -use super::object_store::{ObjectReader, ObjectReaderStream, ObjectStore}; -use super::PartitionedFile; - -/// The base configurations to provide when creating a physical plan for -/// any given file format. -pub struct PhysicalPlanConfig { - /// Store from which the `files` should be fetched - pub object_store: Arc, - /// Schema before projection. It contains the columns that are expected - /// to be in the file followed by the table partition columns. - pub file_schema: SchemaRef, - /// List of files to be processed, grouped into partitions - pub file_groups: Vec>, - /// Estimated overall statistics of the plan, taking `filters` into account - pub statistics: Statistics, - /// Columns on which to project the data - pub projection: Option>, - /// The maximum number of records per arrow column - pub batch_size: usize, - /// The minimum number of records required from this source plan - pub limit: Option, - /// The partitioning column names - pub table_partition_cols: Vec, -} +use super::object_store::{ObjectReader, ObjectReaderStream}; /// This trait abstracts all the file format specific implementations /// from the `TableProvider`. This helps code re-utilization accross diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs index cec9d6762d70..88d658ade92c 100644 --- a/datafusion/src/datasource/listing/helpers.rs +++ b/datafusion/src/datasource/listing/helpers.rs @@ -136,6 +136,7 @@ pub async fn pruned_partition_list( /// - one column for with the original path named `_df_part_file_path_` /// - one column for with the last modified date named `_df_part_file_modified_` /// - ... one column by partition ... +/// /// Note: For the last modified date, this looses precisions higher than millisecond. fn paths_to_batch( table_partition_cols: &[String], diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index 2361c24ae63e..f49bd792da92 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -19,22 +19,23 @@ use std::{any::Any, sync::Arc}; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::datatypes::{Field, Schema, SchemaRef}; use async_trait::async_trait; use futures::StreamExt; use crate::{ error::Result, logical_plan::Expr, - physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}, + physical_plan::{ + empty::EmptyExec, + file_format::{PhysicalPlanConfig, DEFAULT_PARTITION_COLUMN_DATATYPE}, + ExecutionPlan, Statistics, + }, }; use crate::datasource::{ - datasource::TableProviderFilterPushDown, - file_format::{FileFormat, PhysicalPlanConfig}, - get_statistics_with_limit, - object_store::ObjectStore, - PartitionedFile, TableProvider, + datasource::TableProviderFilterPushDown, file_format::FileFormat, + get_statistics_with_limit, object_store::ObjectStore, PartitionedFile, TableProvider, }; use super::helpers::{pruned_partition_list, split_files}; @@ -51,7 +52,8 @@ pub struct ListingOptions { /// partitioning expected should be named "a" and "b": /// - If there is a third level of partitioning it will be ignored. /// - Files that don't follow this partitioning will be ignored. - /// Note that only `DataType::Utf8` is supported for the column type. + /// Note that only `DEFAULT_PARTITION_COLUMN_DATATYPE` is currently + /// supported for the column type. pub table_partition_cols: Vec, /// Set true to try to guess statistics from the files. /// This can add a lot of overhead as it will usually require files @@ -104,8 +106,10 @@ impl ListingOptions { pub struct ListingTable { object_store: Arc, table_path: String, + /// File fields only + file_schema: SchemaRef, /// File fields + partition columns - schema: SchemaRef, + table_schema: SchemaRef, options: ListingOptions, } @@ -117,19 +121,24 @@ impl ListingTable { pub fn new( object_store: Arc, table_path: String, - schema: SchemaRef, + file_schema: SchemaRef, options: ListingOptions, ) -> Self { // Add the partition columns to the file schema - let mut fields = schema.fields().clone(); + let mut table_fields = file_schema.fields().clone(); for part in &options.table_partition_cols { - fields.push(Field::new(part, DataType::Utf8, false)); + table_fields.push(Field::new( + part, + DEFAULT_PARTITION_COLUMN_DATATYPE.clone(), + false, + )); } Self { object_store, table_path, - schema: Arc::new(Schema::new(fields)), + file_schema, + table_schema: Arc::new(Schema::new(table_fields)), options, } } @@ -155,7 +164,7 @@ impl TableProvider for ListingTable { } fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) + Arc::clone(&self.table_schema) } async fn scan( @@ -186,7 +195,7 @@ impl TableProvider for ListingTable { .create_physical_plan( PhysicalPlanConfig { object_store: Arc::clone(&self.object_store), - file_schema: self.schema(), + file_schema: Arc::clone(&self.file_schema), file_groups: partitioned_file_lists, statistics, projection: projection.clone(), @@ -254,6 +263,8 @@ impl ListingTable { #[cfg(test)] mod tests { + use arrow::datatypes::DataType; + use crate::{ datasource::{ file_format::{avro::AvroFormat, parquet::ParquetFormat}, diff --git a/datafusion/src/physical_optimizer/repartition.rs b/datafusion/src/physical_optimizer/repartition.rs index 28b309589149..8ac9dadd9548 100644 --- a/datafusion/src/physical_optimizer/repartition.rs +++ b/datafusion/src/physical_optimizer/repartition.rs @@ -109,9 +109,8 @@ mod tests { use arrow::datatypes::Schema; use super::*; - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::PartitionedFile; - use crate::physical_plan::file_format::ParquetExec; + use crate::physical_plan::file_format::{ParquetExec, PhysicalPlanConfig}; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::Statistics; use crate::test::object_store::TestObjectStore; diff --git a/datafusion/src/physical_plan/coalesce_partitions.rs b/datafusion/src/physical_plan/coalesce_partitions.rs index 285daeac29e4..9c133def8209 100644 --- a/datafusion/src/physical_plan/coalesce_partitions.rs +++ b/datafusion/src/physical_plan/coalesce_partitions.rs @@ -206,9 +206,8 @@ mod tests { use futures::FutureExt; use super::*; - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; - use crate::physical_plan::file_format::CsvExec; + use crate::physical_plan::file_format::{CsvExec, PhysicalPlanConfig}; use crate::physical_plan::{collect, common}; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending}; diff --git a/datafusion/src/physical_plan/file_format/avro.rs b/datafusion/src/physical_plan/file_format/avro.rs index 301b0c3a4dbd..b50c0a082686 100644 --- a/datafusion/src/physical_plan/file_format/avro.rs +++ b/datafusion/src/physical_plan/file_format/avro.rs @@ -18,9 +18,6 @@ //! Execution plan for reading line-delimited Avro files #[cfg(feature = "avro")] use crate::avro_to_arrow; -use crate::datasource::file_format::PhysicalPlanConfig; -use crate::datasource::object_store::ObjectStore; -use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -35,65 +32,30 @@ use std::sync::Arc; #[cfg(feature = "avro")] use super::file_stream::{BatchIter, FileStream}; +use super::PhysicalPlanConfig; /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] pub struct AvroExec { - object_store: Arc, - file_groups: Vec>, - statistics: Statistics, - file_schema: SchemaRef, - projection: Option>, + base_config: PhysicalPlanConfig, + projected_statistics: Statistics, projected_schema: SchemaRef, - batch_size: usize, - limit: Option, - table_partition_cols: Vec, } impl AvroExec { - /// Create a new Avro reader execution plan provided file list and schema + /// Create a new Avro reader execution plan provided base configurations pub fn new(base_config: PhysicalPlanConfig) -> Self { - let (projected_schema, projected_statistics) = super::project( - &base_config.projection, - Arc::clone(&base_config.file_schema), - base_config.statistics, - ); + let (projected_schema, projected_statistics) = base_config.project(); Self { - object_store: base_config.object_store, - file_groups: base_config.file_groups, - statistics: projected_statistics, - file_schema: base_config.file_schema, - projection: base_config.projection, + base_config, projected_schema, - batch_size: base_config.batch_size, - limit: base_config.limit, - table_partition_cols: base_config.table_partition_cols, + projected_statistics, } } - /// List of data files - pub fn file_groups(&self) -> &[Vec] { - &self.file_groups - } - /// The schema before projection - pub fn file_schema(&self) -> &SchemaRef { - &self.file_schema - } - /// Optional projection for which columns to load - pub fn projection(&self) -> &Option> { - &self.projection - } - /// Batch size - pub fn batch_size(&self) -> usize { - self.batch_size - } - /// Limit in nr. of rows - pub fn limit(&self) -> Option { - self.limit - } - /// Partitioning column names - pub fn table_partition_cols(&self) -> &[String] { - &self.table_partition_cols + /// Ref to the base configs + pub fn base_config(&self) -> &PhysicalPlanConfig { + &self.base_config } } @@ -108,7 +70,7 @@ impl ExecutionPlan for AvroExec { } fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.file_groups.len()) + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } fn children(&self) -> Vec> { @@ -138,15 +100,10 @@ impl ExecutionPlan for AvroExec { #[cfg(feature = "avro")] async fn execute(&self, partition: usize) -> Result { - let proj = self.projection.as_ref().map(|p| { - p.iter() - .map(|col_idx| self.file_schema.field(*col_idx).name()) - .cloned() - .collect() - }); + let proj = self.base_config.projected_file_column_names(); - let batch_size = self.batch_size; - let file_schema = Arc::clone(&self.file_schema); + let batch_size = self.base_config.batch_size; + let file_schema = Arc::clone(&self.base_config.file_schema); // The avro reader cannot limit the number of records, so `remaining` is ignored. let fun = move |file, _remaining: &Option| { @@ -165,12 +122,12 @@ impl ExecutionPlan for AvroExec { }; Ok(Box::pin(FileStream::new( - Arc::clone(&self.object_store), - self.file_groups[partition].clone(), + Arc::clone(&self.base_config.object_store), + self.base_config.file_groups[partition].clone(), fun, Arc::clone(&self.projected_schema), - self.limit, - self.table_partition_cols.clone(), + self.base_config.limit, + self.base_config.table_partition_cols.clone(), ))) } @@ -184,16 +141,16 @@ impl ExecutionPlan for AvroExec { write!( f, "AvroExec: files={}, batch_size={}, limit={:?}", - super::FileGroupsDisplay(&self.file_groups), - self.batch_size, - self.limit, + super::FileGroupsDisplay(&self.base_config.file_groups), + self.base_config.batch_size, + self.base_config.limit, ) } } } fn statistics(&self) -> Statistics { - self.statistics.clone() + self.projected_statistics.clone() } } @@ -201,18 +158,17 @@ impl ExecutionPlan for AvroExec { #[cfg(feature = "avro")] mod tests { + use crate::datasource::file_format::{avro::AvroFormat, FileFormat}; use crate::datasource::object_store::local::{ local_object_reader_stream, local_unpartitioned_file, LocalFileSystem, }; + use crate::scalar::ScalarValue; + use futures::StreamExt; use super::*; #[tokio::test] - async fn test() -> Result<()> { - use futures::StreamExt; - - use crate::datasource::file_format::{avro::AvroFormat, FileFormat}; - + async fn avro_exec_without_partition() -> Result<()> { let testdata = crate::test_util::arrow_test_data(); let filename = format!("{}/avro/alltypes_plain.avro", testdata); let avro_exec = AvroExec::new(PhysicalPlanConfig { @@ -229,16 +185,32 @@ mod tests { }); assert_eq!(avro_exec.output_partitioning().partition_count(), 1); - let mut results = avro_exec.execute(0).await?; - let batch = results.next().await.unwrap()?; + let mut results = avro_exec.execute(0).await.expect("plan execution failed"); + let batch = results + .next() + .await + .expect("plan iterator empty") + .expect("plan iterator returned an error"); + + let expected = vec![ + "+----+----------+-------------+", + "| id | bool_col | tinyint_col |", + "+----+----------+-------------+", + "| 4 | true | 0 |", + "| 5 | false | 1 |", + "| 6 | true | 0 |", + "| 7 | false | 1 |", + "| 2 | true | 0 |", + "| 3 | false | 1 |", + "| 0 | true | 0 |", + "| 1 | false | 1 |", + "+----+----------+-------------+", + ]; - assert_eq!(8, batch.num_rows()); - assert_eq!(3, batch.num_columns()); + crate::assert_batches_eq!(expected, &[batch]); - let schema = batch.schema(); - let field_names: Vec<&str> = - schema.fields().iter().map(|f| f.name().as_str()).collect(); - assert_eq!(vec!["id", "bool_col", "tinyint_col"], field_names); + let batch = results.next().await; + assert!(batch.is_none()); let batch = results.next().await; assert!(batch.is_none()); @@ -246,6 +218,57 @@ mod tests { let batch = results.next().await; assert!(batch.is_none()); + Ok(()) + } + + #[tokio::test] + async fn avro_exec_with_partition() -> Result<()> { + let testdata = crate::test_util::arrow_test_data(); + let filename = format!("{}/avro/alltypes_plain.avro", testdata); + let mut partitioned_file = local_unpartitioned_file(filename.clone()); + partitioned_file.partition_values = + vec![ScalarValue::Utf8(Some("2021-10-26".to_owned()))]; + let file_schema = AvroFormat {} + .infer_schema(local_object_reader_stream(vec![filename])) + .await?; + + let avro_exec = AvroExec::new(PhysicalPlanConfig { + // select specific columns of the files as well as the partitioning + // column which is supposed to be the last column in the table schema. + projection: Some(vec![0, 1, file_schema.fields().len(), 2]), + object_store: Arc::new(LocalFileSystem {}), + file_groups: vec![vec![partitioned_file]], + file_schema: file_schema, + statistics: Statistics::default(), + batch_size: 1024, + limit: None, + table_partition_cols: vec!["date".to_owned()], + }); + assert_eq!(avro_exec.output_partitioning().partition_count(), 1); + + let mut results = avro_exec.execute(0).await.expect("plan execution failed"); + let batch = results + .next() + .await + .expect("plan iterator empty") + .expect("plan iterator returned an error"); + + let expected = vec![ + "+----+----------+------------+-------------+", + "| id | bool_col | date | tinyint_col |", + "+----+----------+------------+-------------+", + "| 4 | true | 2021-10-26 | 0 |", + "| 5 | false | 2021-10-26 | 1 |", + "| 6 | true | 2021-10-26 | 0 |", + "| 7 | false | 2021-10-26 | 1 |", + "| 2 | true | 2021-10-26 | 0 |", + "| 3 | false | 2021-10-26 | 1 |", + "| 0 | true | 2021-10-26 | 0 |", + "| 1 | false | 2021-10-26 | 1 |", + "+----+----------+------------+-------------+", + ]; + crate::assert_batches_eq!(expected, &[batch]); + let batch = results.next().await; assert!(batch.is_none()); diff --git a/datafusion/src/physical_plan/file_format/csv.rs b/datafusion/src/physical_plan/file_format/csv.rs index a96d8f314f24..0057e9e811ab 100644 --- a/datafusion/src/physical_plan/file_format/csv.rs +++ b/datafusion/src/physical_plan/file_format/csv.rs @@ -17,9 +17,6 @@ //! Execution plan for reading CSV files -use crate::datasource::file_format::PhysicalPlanConfig; -use crate::datasource::object_store::ObjectStore; -use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -33,54 +30,35 @@ use std::sync::Arc; use async_trait::async_trait; use super::file_stream::{BatchIter, FileStream}; +use super::PhysicalPlanConfig; /// Execution plan for scanning a CSV file #[derive(Debug, Clone)] pub struct CsvExec { - object_store: Arc, - file_groups: Vec>, - file_schema: SchemaRef, + base_config: PhysicalPlanConfig, + projected_statistics: Statistics, projected_schema: SchemaRef, - statistics: Statistics, has_header: bool, delimiter: u8, - projection: Option>, - batch_size: usize, - limit: Option, - table_partition_cols: Vec, } impl CsvExec { - /// Create a new CSV reader execution plan provided file list and schema + /// Create a new CSV reader execution plan provided base and specific configurations pub fn new(base_config: PhysicalPlanConfig, has_header: bool, delimiter: u8) -> Self { - let (projected_schema, projected_statistics) = super::project( - &base_config.projection, - Arc::clone(&base_config.file_schema), - base_config.statistics, - ); + let (projected_schema, projected_statistics) = base_config.project(); Self { - object_store: base_config.object_store, - file_groups: base_config.file_groups, - file_schema: base_config.file_schema, - statistics: projected_statistics, + base_config, + projected_schema, + projected_statistics, has_header, delimiter, - projection: base_config.projection, - projected_schema, - batch_size: base_config.batch_size, - limit: base_config.limit, - table_partition_cols: base_config.table_partition_cols, } } - /// List of data files - pub fn file_groups(&self) -> &[Vec] { - &self.file_groups - } - /// The schema before projection - pub fn file_schema(&self) -> &SchemaRef { - &self.file_schema + /// Ref to the base configs + pub fn base_config(&self) -> &PhysicalPlanConfig { + &self.base_config } /// true if the first line of each file is a header pub fn has_header(&self) -> bool { @@ -90,22 +68,6 @@ impl CsvExec { pub fn delimiter(&self) -> u8 { self.delimiter } - /// Optional projection for which columns to load - pub fn projection(&self) -> &Option> { - &self.projection - } - /// Batch size - pub fn batch_size(&self) -> usize { - self.batch_size - } - /// Limit in nr. of rows - pub fn limit(&self) -> Option { - self.limit - } - /// Partitioning column names - pub fn table_partition_cols(&self) -> &[String] { - &self.table_partition_cols - } } #[async_trait] @@ -122,7 +84,7 @@ impl ExecutionPlan for CsvExec { /// Get the output partitioning of this plan fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.file_groups.len()) + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } fn children(&self) -> Vec> { @@ -145,9 +107,9 @@ impl ExecutionPlan for CsvExec { } async fn execute(&self, partition: usize) -> Result { - let batch_size = self.batch_size; - let file_schema = Arc::clone(&self.file_schema); - let projection = self.projection.clone(); + let batch_size = self.base_config.batch_size; + let file_schema = Arc::clone(&self.base_config.file_schema); + let file_projection = self.base_config.file_column_projection_indices(); let has_header = self.has_header; let delimiter = self.delimiter; let start_line = if has_header { 1 } else { 0 }; @@ -161,17 +123,17 @@ impl ExecutionPlan for CsvExec { Some(delimiter), batch_size, bounds, - projection.clone(), + file_projection.clone(), )) as BatchIter }; Ok(Box::pin(FileStream::new( - Arc::clone(&self.object_store), - self.file_groups[partition].clone(), + Arc::clone(&self.base_config.object_store), + self.base_config.file_groups[partition].clone(), fun, Arc::clone(&self.projected_schema), - self.limit, - self.table_partition_cols.clone(), + self.base_config.limit, + self.base_config.table_partition_cols.clone(), ))) } @@ -185,17 +147,17 @@ impl ExecutionPlan for CsvExec { write!( f, "CsvExec: files={}, has_header={}, batch_size={}, limit={:?}", - super::FileGroupsDisplay(&self.file_groups), + super::FileGroupsDisplay(&self.base_config.file_groups), self.has_header, - self.batch_size, - self.limit, + self.base_config.batch_size, + self.base_config.limit, ) } } } fn statistics(&self) -> Statistics { - self.statistics.clone() + self.projected_statistics.clone() } } @@ -204,6 +166,7 @@ mod tests { use super::*; use crate::{ datasource::object_store::local::{local_unpartitioned_file, LocalFileSystem}, + scalar::ScalarValue, test::aggr_test_schema, }; use futures::StreamExt; @@ -228,22 +191,34 @@ mod tests { true, b',', ); - assert_eq!(13, csv.file_schema.fields().len()); + assert_eq!(13, csv.base_config.file_schema.fields().len()); assert_eq!(3, csv.projected_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); + let mut stream = csv.execute(0).await?; let batch = stream.next().await.unwrap()?; assert_eq!(3, batch.num_columns()); - let batch_schema = batch.schema(); - assert_eq!(3, batch_schema.fields().len()); - assert_eq!("c1", batch_schema.field(0).name()); - assert_eq!("c3", batch_schema.field(1).name()); - assert_eq!("c5", batch_schema.field(2).name()); + assert_eq!(100, batch.num_rows()); + + // slice of the first 5 lines + let expected = vec![ + "+----+-----+------------+", + "| c1 | c3 | c5 |", + "+----+-----+------------+", + "| c | 1 | 2033001162 |", + "| d | -40 | 706441268 |", + "| b | 29 | 994303988 |", + "| a | -85 | 1171968280 |", + "| b | -82 | 1824882165 |", + "+----+-----+------------+", + ]; + + crate::assert_batches_eq!(expected, &[batch.slice(0, 5)]); Ok(()) } #[tokio::test] - async fn csv_exec_without_projection() -> Result<()> { + async fn csv_exec_with_limit() -> Result<()> { let file_schema = aggr_test_schema(); let testdata = crate::test_util::arrow_test_data(); let filename = "aggregate_test_100.csv"; @@ -256,23 +231,87 @@ mod tests { statistics: Statistics::default(), projection: None, batch_size: 1024, - limit: None, + limit: Some(5), table_partition_cols: vec![], }, true, b',', ); - assert_eq!(13, csv.file_schema.fields().len()); + assert_eq!(13, csv.base_config.file_schema.fields().len()); assert_eq!(13, csv.projected_schema.fields().len()); assert_eq!(13, csv.schema().fields().len()); + let mut it = csv.execute(0).await?; let batch = it.next().await.unwrap()?; assert_eq!(13, batch.num_columns()); - let batch_schema = batch.schema(); - assert_eq!(13, batch_schema.fields().len()); - assert_eq!("c1", batch_schema.field(0).name()); - assert_eq!("c2", batch_schema.field(1).name()); - assert_eq!("c3", batch_schema.field(2).name()); + assert_eq!(5, batch.num_rows()); + + let expected = vec![ + "+----+----+-----+--------+------------+----------------------+-----+-------+------------+----------------------+-------------+---------------------+--------------------------------+", + "| c1 | c2 | c3 | c4 | c5 | c6 | c7 | c8 | c9 | c10 | c11 | c12 | c13 |", + "+----+----+-----+--------+------------+----------------------+-----+-------+------------+----------------------+-------------+---------------------+--------------------------------+", + "| c | 2 | 1 | 18109 | 2033001162 | -6513304855495910254 | 25 | 43062 | 1491205016 | 5863949479783605708 | 0.110830784 | 0.9294097332465232 | 6WfVFBVGJSQb7FhA7E0lBwdvjfZnSW |", + "| d | 5 | -40 | 22614 | 706441268 | -7542719935673075327 | 155 | 14337 | 3373581039 | 11720144131976083864 | 0.69632107 | 0.3114712539863804 | C2GT5KVyOPZpgKVl110TyZO0NcJ434 |", + "| b | 1 | 29 | -18218 | 994303988 | 5983957848665088916 | 204 | 9489 | 3275293996 | 14857091259186476033 | 0.53840446 | 0.17909035118828576 | AyYVExXK6AR2qUTxNZ7qRHQOVGMLcz |", + "| a | 1 | -85 | -15154 | 1171968280 | 1919439543497968449 | 77 | 52286 | 774637006 | 12101411955859039553 | 0.12285209 | 0.6864391962767343 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB |", + "| b | 5 | -82 | 22080 | 1824882165 | 7373730676428214987 | 208 | 34331 | 3342719438 | 3330177516592499461 | 0.82634634 | 0.40975383525297016 | Ig1QcuKsjHXkproePdERo2w0mYzIqd |", + "+----+----+-----+--------+------------+----------------------+-----+-------+------------+----------------------+-------------+---------------------+--------------------------------+", + ]; + + crate::assert_batches_eq!(expected, &[batch]); + + Ok(()) + } + + #[tokio::test] + async fn csv_exec_with_partition() -> Result<()> { + let file_schema = aggr_test_schema(); + let testdata = crate::test_util::arrow_test_data(); + let filename = "aggregate_test_100.csv"; + // we don't have `/date=xx/` in the path but that is ok because + // partitions are resolved during scan anyway + let path = format!("{}/csv/{}", testdata, filename); + let mut partitioned_file = local_unpartitioned_file(path); + partitioned_file.partition_values = + vec![ScalarValue::Utf8(Some("2021-10-26".to_owned()))]; + let csv = CsvExec::new( + PhysicalPlanConfig { + // we should be able to project on the partition column + // wich is supposed to be after the file fields + projection: Some(vec![0, file_schema.fields().len()]), + object_store: Arc::new(LocalFileSystem {}), + file_schema, + file_groups: vec![vec![partitioned_file]], + statistics: Statistics::default(), + batch_size: 1024, + limit: None, + table_partition_cols: vec!["date".to_owned()], + }, + true, + b',', + ); + assert_eq!(13, csv.base_config.file_schema.fields().len()); + assert_eq!(2, csv.projected_schema.fields().len()); + assert_eq!(2, csv.schema().fields().len()); + + let mut it = csv.execute(0).await?; + let batch = it.next().await.unwrap()?; + assert_eq!(2, batch.num_columns()); + assert_eq!(100, batch.num_rows()); + + // slice of the first 5 lines + let expected = vec![ + "+----+------------+", + "| c1 | date |", + "+----+------------+", + "| c | 2021-10-26 |", + "| d | 2021-10-26 |", + "| b | 2021-10-26 |", + "| a | 2021-10-26 |", + "| b | 2021-10-26 |", + "+----+------------+", + ]; + crate::assert_batches_eq!(expected, &[batch.slice(0, 5)]); Ok(()) } } diff --git a/datafusion/src/physical_plan/file_format/file_stream.rs b/datafusion/src/physical_plan/file_format/file_stream.rs index 5c2c4525b0a0..958b1721bb39 100644 --- a/datafusion/src/physical_plan/file_format/file_stream.rs +++ b/datafusion/src/physical_plan/file_format/file_stream.rs @@ -27,15 +27,12 @@ use crate::{ scalar::ScalarValue, }; use arrow::{ - array::{ArrayData, ArrayRef, DictionaryArray, UInt8BufferBuilder}, - buffer::Buffer, - datatypes::{DataType, SchemaRef, UInt8Type}, + datatypes::SchemaRef, error::{ArrowError, Result as ArrowResult}, record_batch::RecordBatch, }; use futures::Stream; use std::{ - collections::HashMap, io::Read, iter, pin::Pin, @@ -43,6 +40,8 @@ use std::{ task::{Context, Poll}, }; +use super::PartitionColumnProjector; + pub type FileIter = Box + Send + Sync>; pub type BatchIter = Box> + Send + Sync>; @@ -71,7 +70,7 @@ pub struct FileStream { file_iter: FileIter, /// The stream schema (file schema including partition columns and after /// projection). - schema: SchemaRef, + projected_schema: SchemaRef, /// The remaining number of records to parse, None if no limit remain: Option, /// A closure that takes a reader and an optional remaining number of lines @@ -79,13 +78,8 @@ pub struct FileStream { /// is not capable of limiting the number of records in the last batch, the file /// stream will take care of truncating it. file_reader: F, - /// A buffer initialized to zeros that represents the key array of all partition - /// columns (partition columns are materialized by dictionary arrays with only one - /// value in the dictionary, thus all the keys are equal to zero). - key_buffer_cache: Option, - /// mapping between the indexes in the list of partition columns and the target - /// schema. - projected_partition_indexes: HashMap, + /// The partition column projector + pc_projector: PartitionColumnProjector, /// the store from which to source the files. object_store: Arc, } @@ -95,26 +89,23 @@ impl FileStream { object_store: Arc, files: Vec, file_reader: F, - schema: SchemaRef, + projected_schema: SchemaRef, limit: Option, table_partition_cols: Vec, ) -> Self { - let mut projected_partition_indexes = HashMap::new(); - for (partition_idx, partition_name) in table_partition_cols.iter().enumerate() { - if let Ok(schema_idx) = schema.index_of(partition_name) { - projected_partition_indexes.insert(partition_idx, schema_idx); - } - } + let pc_projector = PartitionColumnProjector::new( + Arc::clone(&projected_schema), + &table_partition_cols, + ); Self { file_iter: Box::new(files.into_iter()), batch_iter: Box::new(iter::empty()), partition_values: vec![], remain: limit, - schema, + projected_schema, file_reader, - key_buffer_cache: None, - projected_partition_indexes, + pc_projector, object_store, } } @@ -122,28 +113,10 @@ impl FileStream { /// Acts as a flat_map of record batches over files. Adds the partitioning /// Columns to the returned record batches. fn next_batch(&mut self) -> Option> { - let expected_cols = - self.schema.fields().len() - self.projected_partition_indexes.len(); match self.batch_iter.next() { - Some(Ok(batch)) if batch.columns().len() == expected_cols => { - let mut cols = batch.columns().to_vec(); - for (&pidx, &sidx) in &self.projected_partition_indexes { - cols.insert( - sidx, - create_dict_array( - &mut self.key_buffer_cache, - &self.partition_values[pidx], - batch.num_rows(), - ), - ) - } - Some(RecordBatch::try_new(self.schema(), cols)) + Some(Ok(batch)) => { + Some(self.pc_projector.project(batch, &self.partition_values)) } - Some(Ok(batch)) => Some(Err(ArrowError::SchemaError(format!( - "Unexpected batch schema from file, expected {} cols but got {}", - expected_cols, - batch.columns().len() - )))), Some(Err(e)) => Some(Err(e)), None => match self.file_iter.next() { Some(f) => { @@ -164,36 +137,6 @@ impl FileStream { } } -fn create_dict_array( - key_buffer_cache: &mut Option, - val: &ScalarValue, - len: usize, -) -> ArrayRef { - // build value dictionary - let dict_vals = val.to_array(); - - // build keys array - let sliced_key_buffer = match key_buffer_cache { - Some(buf) if buf.len() >= len => buf.slice(buf.len() - len), - _ => { - let mut key_buffer_builder = UInt8BufferBuilder::new(len); - key_buffer_builder.advance(len); // keys are all 0 - key_buffer_cache.insert(key_buffer_builder.finish()).clone() - } - }; - - // create data type - let data_type = - DataType::Dictionary(Box::new(DataType::UInt8), Box::new(val.get_datatype())); - - // assemble pieces together - let mut builder = ArrayData::builder(data_type) - .len(len) - .add_buffer(sliced_key_buffer); - builder = builder.add_child_data(dict_vals.data().clone()); - Arc::new(DictionaryArray::::from(builder.build().unwrap())) -} - impl Stream for FileStream { type Item = ArrowResult; @@ -236,7 +179,7 @@ impl Stream for FileStream { impl RecordBatchStream for FileStream { fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) + Arc::clone(&self.projected_schema) } } diff --git a/datafusion/src/physical_plan/file_format/json.rs b/datafusion/src/physical_plan/file_format/json.rs index d1309fab938d..9032eb9d5e5d 100644 --- a/datafusion/src/physical_plan/file_format/json.rs +++ b/datafusion/src/physical_plan/file_format/json.rs @@ -18,9 +18,6 @@ //! Execution plan for reading line-delimited JSON files use async_trait::async_trait; -use crate::datasource::file_format::PhysicalPlanConfig; -use crate::datasource::object_store::ObjectStore; -use crate::datasource::PartitionedFile; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -30,40 +27,25 @@ use std::any::Any; use std::sync::Arc; use super::file_stream::{BatchIter, FileStream}; +use super::PhysicalPlanConfig; /// Execution plan for scanning NdJson data source #[derive(Debug, Clone)] pub struct NdJsonExec { - object_store: Arc, - file_groups: Vec>, - statistics: Statistics, - file_schema: SchemaRef, - projection: Option>, + base_config: PhysicalPlanConfig, + projected_statistics: Statistics, projected_schema: SchemaRef, - batch_size: usize, - limit: Option, - table_partition_cols: Vec, } impl NdJsonExec { - /// Create a new JSON reader execution plan provided file list and schema + /// Create a new JSON reader execution plan provided base configurations pub fn new(base_config: PhysicalPlanConfig) -> Self { - let (projected_schema, projected_statistics) = super::project( - &base_config.projection, - Arc::clone(&base_config.file_schema), - base_config.statistics, - ); + let (projected_schema, projected_statistics) = base_config.project(); Self { - object_store: base_config.object_store, - file_groups: base_config.file_groups, - statistics: projected_statistics, - file_schema: base_config.file_schema, - projection: base_config.projection, + base_config, projected_schema, - batch_size: base_config.batch_size, - limit: base_config.limit, - table_partition_cols: base_config.table_partition_cols, + projected_statistics, } } } @@ -79,7 +61,7 @@ impl ExecutionPlan for NdJsonExec { } fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.file_groups.len()) + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } fn children(&self) -> Vec> { @@ -101,15 +83,10 @@ impl ExecutionPlan for NdJsonExec { } async fn execute(&self, partition: usize) -> Result { - let proj = self.projection.as_ref().map(|p| { - p.iter() - .map(|col_idx| self.file_schema.field(*col_idx).name()) - .cloned() - .collect() - }); + let proj = self.base_config.projected_file_column_names(); - let batch_size = self.batch_size; - let file_schema = Arc::clone(&self.file_schema); + let batch_size = self.base_config.batch_size; + let file_schema = Arc::clone(&self.base_config.file_schema); // The json reader cannot limit the number of records, so `remaining` is ignored. let fun = move |file, _remaining: &Option| { @@ -122,12 +99,12 @@ impl ExecutionPlan for NdJsonExec { }; Ok(Box::pin(FileStream::new( - Arc::clone(&self.object_store), - self.file_groups[partition].clone(), + Arc::clone(&self.base_config.object_store), + self.base_config.file_groups[partition].clone(), fun, Arc::clone(&self.projected_schema), - self.limit, - self.table_partition_cols.clone(), + self.base_config.limit, + self.base_config.table_partition_cols.clone(), ))) } @@ -141,16 +118,16 @@ impl ExecutionPlan for NdJsonExec { write!( f, "JsonExec: batch_size={}, limit={:?}, files={}", - self.batch_size, - self.limit, - super::FileGroupsDisplay(&self.file_groups), + self.base_config.batch_size, + self.base_config.limit, + super::FileGroupsDisplay(&self.base_config.file_groups), ) } } } fn statistics(&self) -> Statistics { - self.statistics.clone() + self.projected_statistics.clone() } } diff --git a/datafusion/src/physical_plan/file_format/mod.rs b/datafusion/src/physical_plan/file_format/mod.rs index be84151201c9..64788b9bc046 100644 --- a/datafusion/src/physical_plan/file_format/mod.rs +++ b/datafusion/src/physical_plan/file_format/mod.rs @@ -24,18 +24,127 @@ mod json; mod parquet; pub use self::parquet::ParquetExec; -use arrow::datatypes::{Schema, SchemaRef}; +use arrow::{ + array::{ArrayData, ArrayRef, DictionaryArray, UInt8BufferBuilder}, + buffer::Buffer, + datatypes::{DataType, Field, Schema, SchemaRef, UInt8Type}, + error::{ArrowError, Result as ArrowResult}, + record_batch::RecordBatch, +}; pub use avro::AvroExec; pub use csv::CsvExec; pub use json::NdJsonExec; -use crate::datasource::PartitionedFile; +use crate::{ + datasource::{object_store::ObjectStore, PartitionedFile}, + scalar::ScalarValue, +}; use std::{ + collections::HashMap, fmt::{Display, Formatter, Result as FmtResult}, sync::Arc, + vec, }; -use super::Statistics; +use super::{ColumnStatistics, Statistics}; + +lazy_static! { + /// The datatype used for all partitioning columns for now + pub static ref DEFAULT_PARTITION_COLUMN_DATATYPE: DataType = DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)); +} + +/// The base configurations to provide when creating a physical plan for +/// any given file format. +#[derive(Debug, Clone)] +pub struct PhysicalPlanConfig { + /// Store from which the `files` should be fetched + pub object_store: Arc, + /// Schema before projection. It contains the columns that are expected + /// to be in the files without the table partition columns. + pub file_schema: SchemaRef, + /// List of files to be processed, grouped into partitions + pub file_groups: Vec>, + /// Estimated overall statistics of the files, taking `filters` into account. + pub statistics: Statistics, + /// Columns on which to project the data. Indexes that are higher than the + /// number of columns of `file_schema` refer to `table_partition_cols`. + pub projection: Option>, + /// The maximum number of records per arrow column + pub batch_size: usize, + /// The minimum number of records required from this source plan + pub limit: Option, + /// The partitioning column names + pub table_partition_cols: Vec, +} + +impl PhysicalPlanConfig { + /// Project the schema and the statistics on the given column indices + fn project(&self) -> (SchemaRef, Statistics) { + if self.projection.is_none() && self.table_partition_cols.is_empty() { + return (Arc::clone(&self.file_schema), self.statistics.clone()); + } + + let proj_iter: Box> = match &self.projection { + Some(proj) => Box::new(proj.iter().copied()), + None => Box::new( + 0..(self.file_schema.fields().len() + self.table_partition_cols.len()), + ), + }; + + let mut table_fields = vec![]; + let mut table_cols_stats = vec![]; + for idx in proj_iter { + if idx < self.file_schema.fields().len() { + table_fields.push(self.file_schema.field(idx).clone()); + if let Some(file_cols_stats) = &self.statistics.column_statistics { + table_cols_stats.push(file_cols_stats[idx].clone()) + } else { + table_cols_stats.push(ColumnStatistics::default()) + } + } else { + let partition_idx = idx - self.file_schema.fields().len(); + table_fields.push(Field::new( + &self.table_partition_cols[partition_idx], + DEFAULT_PARTITION_COLUMN_DATATYPE.clone(), + false, + )); + // TODO provide accurate stat for partition column + table_cols_stats.push(ColumnStatistics::default()) + } + } + + let table_stats = Statistics { + num_rows: self.statistics.num_rows, + is_exact: self.statistics.is_exact, + // TODO correct byte size? + total_byte_size: None, + column_statistics: Some(table_cols_stats), + }; + + let table_schema = Arc::new(Schema::new(table_fields)); + + (table_schema, table_stats) + } + + fn projected_file_column_names(&self) -> Option> { + self.projection.as_ref().map(|p| { + p.iter() + .filter(|col_idx| **col_idx < self.file_schema.fields().len()) + .map(|col_idx| self.file_schema.field(*col_idx).name()) + .cloned() + .collect() + }) + } + + fn file_column_projection_indices(&self) -> Option> { + self.projection.as_ref().map(|p| { + p.iter() + .filter(|col_idx| **col_idx < self.file_schema.fields().len()) + .copied() + .collect() + }) + } +} /// A wrapper to customize partitioned file display #[derive(Debug)] @@ -57,38 +166,259 @@ impl<'a> Display for FileGroupsDisplay<'a> { } } -/// Project the schema and the statistics on the given column indices -fn project( - projection: &Option>, - schema: SchemaRef, - statistics: Statistics, -) -> (SchemaRef, Statistics) { - let projection = if projection.is_none() { - return (schema, statistics); - } else { - projection.as_ref().unwrap() - }; - let projected_schema = Schema::new( - projection - .iter() - .map(|i| schema.field(*i).clone()) - .collect(), - ); - - let new_column_statistics = statistics.column_statistics.map(|stats| { - let mut projected_stats = Vec::with_capacity(projection.len()); - for proj in projection { - projected_stats.push(stats[*proj].clone()); +/// A helper that projects partition columns into the file record batches +struct PartitionColumnProjector { + /// An Arrow buffer initialized to zeros that represents the key array of all partition + /// columns (partition columns are materialized by dictionary arrays with only one + /// value in the dictionary, thus all the keys are equal to zero). + key_buffer_cache: Option, + /// Mapping between the indexes in the list of partition columns and the target + /// schema. Sorted by index in the target schema so that we can iterate on it to + /// insert the partition columns in the target record batch. + projected_partition_indexes: Vec<(usize, usize)>, + /// The schema of the table once the projection was applied. + projected_schema: SchemaRef, +} + +impl PartitionColumnProjector { + // Create a projector to insert the partitioning columns into batches read from files + // - projected_schema: the target schema with both file and partitioning columns + // - table_partition_cols: all the partitioning column names + fn new(projected_schema: SchemaRef, table_partition_cols: &[String]) -> Self { + let mut idx_map = HashMap::new(); + for (partition_idx, partition_name) in table_partition_cols.iter().enumerate() { + if let Ok(schema_idx) = projected_schema.index_of(partition_name) { + idx_map.insert(partition_idx, schema_idx); + } + } + + let mut projected_partition_indexes: Vec<_> = idx_map.into_iter().collect(); + projected_partition_indexes.sort_by(|(_, a), (_, b)| a.cmp(b)); + + Self { + projected_partition_indexes, + key_buffer_cache: None, + projected_schema, + } + } + + // Transform the batch read from the fileby inserting the partitioning columns + // to the right positions as deduced from `projected_schema` + // - file_batch: batch read from the file, with internal projection applied + // - partition_values: the list of partition values, one for each partition column + fn project( + &mut self, + file_batch: RecordBatch, + partition_values: &[ScalarValue], + ) -> ArrowResult { + let expected_cols = + self.projected_schema.fields().len() - self.projected_partition_indexes.len(); + + if file_batch.columns().len() != expected_cols { + return Err(ArrowError::SchemaError(format!( + "Unexpected batch schema from file, expected {} cols but got {}", + expected_cols, + file_batch.columns().len() + ))); + } + + let mut cols = file_batch.columns().to_vec(); + for &(pidx, sidx) in &self.projected_partition_indexes { + cols.insert( + sidx, + create_dict_array( + &mut self.key_buffer_cache, + &partition_values[pidx], + file_batch.num_rows(), + ), + ) + } + RecordBatch::try_new(Arc::clone(&self.projected_schema), cols) + } +} + +fn create_dict_array( + key_buffer_cache: &mut Option, + val: &ScalarValue, + len: usize, +) -> ArrayRef { + // build value dictionary + let dict_vals = val.to_array(); + + // build keys array + let sliced_key_buffer = match key_buffer_cache { + Some(buf) if buf.len() >= len => buf.slice(buf.len() - len), + _ => { + let mut key_buffer_builder = UInt8BufferBuilder::new(len); + key_buffer_builder.advance(len); // keys are all 0 + key_buffer_cache.insert(key_buffer_builder.finish()).clone() } - projected_stats - }); - - let statistics = Statistics { - num_rows: statistics.num_rows, - total_byte_size: statistics.total_byte_size, - column_statistics: new_column_statistics, - is_exact: statistics.is_exact, }; - (Arc::new(projected_schema), statistics) + // create data type + let data_type = + DataType::Dictionary(Box::new(DataType::UInt8), Box::new(val.get_datatype())); + + debug_assert_eq!(data_type, *DEFAULT_PARTITION_COLUMN_DATATYPE); + + // assemble pieces together + let mut builder = ArrayData::builder(data_type) + .len(len) + .add_buffer(sliced_key_buffer); + builder = builder.add_child_data(dict_vals.data().clone()); + Arc::new(DictionaryArray::::from(builder.build().unwrap())) +} + +#[cfg(test)] +mod tests { + use crate::test::{ + aggr_test_schema, build_table_i32, columns, object_store::TestObjectStore, + }; + + use super::*; + + #[test] + fn physical_plan_config_no_projection() { + let file_schema = aggr_test_schema(); + let conf = config_for_projection( + Arc::clone(&file_schema), + None, + Statistics::default(), + vec!["date".to_owned()], + ); + + let (proj_schema, proj_statistics) = conf.project(); + assert_eq!(proj_schema.fields().len(), file_schema.fields().len() + 1); + assert_eq!( + proj_schema.field(file_schema.fields().len()).name(), + "date", + "partition columns are the last columns" + ); + assert_eq!( + proj_statistics + .column_statistics + .expect("projection creates column statistics") + .len(), + file_schema.fields().len() + 1 + ); + // TODO implement tests for partition column statistics once implemented + + let col_names = conf.projected_file_column_names(); + assert_eq!(col_names, None); + + let col_indices = conf.file_column_projection_indices(); + assert_eq!(col_indices, None); + } + + #[test] + fn physical_plan_config_with_projection() { + let file_schema = aggr_test_schema(); + let conf = config_for_projection( + Arc::clone(&file_schema), + Some(vec![file_schema.fields().len(), 0]), + Statistics { + num_rows: Some(10), + // assign the column index to distinct_count to help assert + // the source statistic after the projection + column_statistics: Some( + (0..file_schema.fields().len()) + .map(|i| ColumnStatistics { + distinct_count: Some(i), + ..Default::default() + }) + .collect(), + ), + ..Default::default() + }, + vec!["date".to_owned()], + ); + + let (proj_schema, proj_statistics) = conf.project(); + assert_eq!( + columns(&proj_schema), + vec!["date".to_owned(), "c1".to_owned()] + ); + let proj_stat_cols = proj_statistics + .column_statistics + .expect("projection creates column statistics"); + assert_eq!(proj_stat_cols.len(), 2); + // TODO implement tests for proj_stat_cols[0] once partition column + // statistics are implemented + assert_eq!(proj_stat_cols[1].distinct_count, Some(0)); + + let col_names = conf.projected_file_column_names(); + assert_eq!(col_names, Some(vec!["c1".to_owned()])); + + let col_indices = conf.file_column_projection_indices(); + assert_eq!(col_indices, Some(vec![0])); + } + + #[test] + fn partition_column_projector() { + let file_batch = build_table_i32( + ("a", &vec![0, 1, 2]), + ("b", &vec![-2, -1, 0]), + ("c", &vec![10, 11, 12]), + ); + let partition_cols = + vec!["year".to_owned(), "month".to_owned(), "day".to_owned()]; + // create a projected schema + let conf = config_for_projection( + file_batch.schema(), + // keep all cols from file and 2 from partitioning + Some(vec![ + 0, + 1, + 2, + file_batch.schema().fields().len(), + file_batch.schema().fields().len() + 2, + ]), + Statistics::default(), + partition_cols.clone(), + ); + let (proj_schema, _) = conf.project(); + // created a projector for that projected schema + let mut proj = PartitionColumnProjector::new(proj_schema, &partition_cols); + let projected_batch = proj + .project( + // file_batch is ok here because we kept all the file cols in the projection + file_batch, + &[ + ScalarValue::Utf8(Some("2021".to_owned())), + ScalarValue::Utf8(Some("10".to_owned())), + ScalarValue::Utf8(Some("26".to_owned())), + ], + ) + .expect("Projection of partition columns into record batch failed"); + + let expected = vec![ + "+---+----+----+------+-----+", + "| a | b | c | year | day |", + "+---+----+----+------+-----+", + "| 0 | -2 | 10 | 2021 | 26 |", + "| 1 | -1 | 11 | 2021 | 26 |", + "| 2 | 0 | 12 | 2021 | 26 |", + "+---+----+----+------+-----+", + ]; + crate::assert_batches_eq!(expected, &[projected_batch]); + } + + // sets default for configs that play no role in projections + fn config_for_projection( + file_schema: SchemaRef, + projection: Option>, + statistics: Statistics, + table_partition_cols: Vec, + ) -> PhysicalPlanConfig { + PhysicalPlanConfig { + batch_size: 1024, + file_schema, + file_groups: vec![vec![]], + limit: None, + object_store: TestObjectStore::new_arc(&[]), + projection, + statistics, + table_partition_cols, + } + } } diff --git a/datafusion/src/physical_plan/file_format/parquet.rs b/datafusion/src/physical_plan/file_format/parquet.rs index 8cdab90e14e8..e7980d9aa6d3 100644 --- a/datafusion/src/physical_plan/file_format/parquet.rs +++ b/datafusion/src/physical_plan/file_format/parquet.rs @@ -22,7 +22,6 @@ use std::sync::Arc; use std::{any::Any, convert::TryInto}; use crate::datasource::file_format::parquet::ChunkObjectReader; -use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::ObjectStore; use crate::datasource::PartitionedFile; use crate::{ @@ -30,6 +29,7 @@ use crate::{ logical_plan::{Column, Expr}, physical_optimizer::pruning::{PruningPredicate, PruningStatistics}, physical_plan::{ + file_format::PhysicalPlanConfig, metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, stream::RecordBatchReceiverStream, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, @@ -61,29 +61,18 @@ use tokio::{ use async_trait::async_trait; +use super::PartitionColumnProjector; + /// Execution plan for scanning one or more Parquet partitions #[derive(Debug, Clone)] pub struct ParquetExec { - object_store: Arc, - /// Grouped list of files. Each group will be processed together by one - /// partition of the `ExecutionPlan`. - file_groups: Vec>, - /// Schema after projection is applied - schema: SchemaRef, - /// Projection for which columns to load - projection: Vec, - /// Batch size - batch_size: usize, - /// Statistics for the data set (sum of statistics for all partitions) - statistics: Statistics, + base_config: PhysicalPlanConfig, + projected_statistics: Statistics, + projected_schema: SchemaRef, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Optional predicate builder predicate_builder: Option, - /// Optional limit of the number of rows - limit: Option, - /// Partioning column names - table_partition_cols: Vec, } /// Stores metrics about the parquet execution for a particular parquet file @@ -123,50 +112,20 @@ impl ParquetExec { } }); - let expanded_projection = match &base_config.projection { - Some(p) => p.to_vec(), - None => (0..base_config.file_schema.fields().len()).collect(), - }; - - let (projected_schema, projected_statistics) = super::project( - &base_config.projection, - base_config.file_schema, - base_config.statistics, - ); + let (projected_schema, projected_statistics) = base_config.project(); Self { - object_store: base_config.object_store, - file_groups: base_config.file_groups, - schema: projected_schema, - projection: expanded_projection, + base_config, + projected_schema, + projected_statistics, metrics, predicate_builder, - batch_size: base_config.batch_size, - statistics: projected_statistics, - limit: base_config.limit, - table_partition_cols: base_config.table_partition_cols, } } - /// List of data files - pub fn file_groups(&self) -> &[Vec] { - &self.file_groups - } - /// Optional projection for which columns to load - pub fn projection(&self) -> &[usize] { - &self.projection - } - /// Batch size - pub fn batch_size(&self) -> usize { - self.batch_size - } - /// Limit in nr. of rows - pub fn limit(&self) -> Option { - self.limit - } - /// Partitioning column names - pub fn table_partition_cols(&self) -> &[String] { - &self.table_partition_cols + /// Ref to the base configs + pub fn base_config(&self) -> &PhysicalPlanConfig { + &self.base_config } } @@ -200,7 +159,7 @@ impl ExecutionPlan for ParquetExec { } fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.projected_schema) } fn children(&self) -> Vec> { @@ -210,7 +169,7 @@ impl ExecutionPlan for ParquetExec { /// Get the output partitioning of this plan fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.file_groups.len()) + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } fn with_new_children( @@ -235,13 +194,20 @@ impl ExecutionPlan for ParquetExec { Receiver>, ) = channel(2); - let partition = self.file_groups[partition_index].clone(); + let partition = self.base_config.file_groups[partition_index].clone(); let metrics = self.metrics.clone(); - let projection = self.projection.clone(); + let projection = match self.base_config.file_column_projection_indices() { + Some(proj) => proj, + None => (0..self.base_config.file_schema.fields().len()).collect(), + }; let predicate_builder = self.predicate_builder.clone(); - let batch_size = self.batch_size; - let limit = self.limit; - let object_store = Arc::clone(&self.object_store); + let batch_size = self.base_config.batch_size; + let limit = self.base_config.limit; + let object_store = Arc::clone(&self.base_config.object_store); + let partition_col_proj = PartitionColumnProjector::new( + Arc::clone(&self.projected_schema), + &self.base_config.table_partition_cols, + ); let join_handle = task::spawn_blocking(move || { if let Err(e) = read_partition( @@ -254,13 +220,14 @@ impl ExecutionPlan for ParquetExec { batch_size, response_tx, limit, + partition_col_proj, ) { println!("Parquet reader thread terminated due to error: {:?}", e); } }); Ok(RecordBatchReceiverStream::create( - &self.schema, + &self.projected_schema, response_rx, join_handle, )) @@ -276,9 +243,9 @@ impl ExecutionPlan for ParquetExec { write!( f, "ParquetExec: batch_size={}, limit={:?}, partitions={}", - self.batch_size, - self.limit, - super::FileGroupsDisplay(&self.file_groups) + self.base_config.batch_size, + self.base_config.limit, + super::FileGroupsDisplay(&self.base_config.file_groups) ) } } @@ -289,7 +256,7 @@ impl ExecutionPlan for ParquetExec { } fn statistics(&self) -> Statistics { - self.statistics.clone() + self.projected_statistics.clone() } } @@ -429,6 +396,7 @@ fn read_partition( batch_size: usize, response_tx: Sender>, limit: Option, + mut partition_column_projector: PartitionColumnProjector, ) -> Result<()> { let mut total_rows = 0; 'outer: for partitioned_file in partition { @@ -456,8 +424,10 @@ fn read_partition( match batch_reader.next() { Some(Ok(batch)) => { total_rows += batch.num_rows(); - // TODO add partitioning columns here! - send_result(&response_tx, Ok(batch))?; + let proj_batch = partition_column_projector + .project(batch, &partitioned_file.partition_values); + + send_result(&response_tx, proj_batch)?; if limit.map(|l| total_rows >= l).unwrap_or(false) { break 'outer; } @@ -507,7 +477,7 @@ mod tests { }; #[tokio::test] - async fn test() -> Result<()> { + async fn parquet_exec_with_projection() -> Result<()> { let testdata = crate::test_util::parquet_test_data(); let filename = format!("{}/alltypes_plain.parquet", testdata); let parquet_exec = ParquetExec::new( @@ -550,6 +520,62 @@ mod tests { Ok(()) } + #[tokio::test] + async fn parquet_exec_with_partition() -> Result<()> { + let testdata = crate::test_util::parquet_test_data(); + let filename = format!("{}/alltypes_plain.parquet", testdata); + let mut partitioned_file = local_unpartitioned_file(filename.clone()); + partitioned_file.partition_values = vec![ + ScalarValue::Utf8(Some("2021".to_owned())), + ScalarValue::Utf8(Some("10".to_owned())), + ScalarValue::Utf8(Some("26".to_owned())), + ]; + let parquet_exec = ParquetExec::new( + PhysicalPlanConfig { + object_store: Arc::new(LocalFileSystem {}), + file_groups: vec![vec![partitioned_file]], + file_schema: ParquetFormat::default() + .infer_schema(local_object_reader_stream(vec![filename])) + .await?, + statistics: Statistics::default(), + // file has 10 cols so index 12 should be month + projection: Some(vec![0, 1, 2, 12]), + batch_size: 1024, + limit: None, + table_partition_cols: vec![ + "year".to_owned(), + "month".to_owned(), + "day".to_owned(), + ], + }, + None, + ); + assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); + + let mut results = parquet_exec.execute(0).await?; + let batch = results.next().await.unwrap()?; + let expected = vec![ + "+----+----------+-------------+-------+", + "| id | bool_col | tinyint_col | month |", + "+----+----------+-------------+-------+", + "| 4 | true | 0 | 10 |", + "| 5 | false | 1 | 10 |", + "| 6 | true | 0 | 10 |", + "| 7 | false | 1 | 10 |", + "| 2 | true | 0 | 10 |", + "| 3 | false | 1 | 10 |", + "| 0 | true | 0 | 10 |", + "| 1 | false | 1 | 10 |", + "+----+----------+-------------+-------+", + ]; + crate::assert_batches_eq!(expected, &[batch]); + + let batch = results.next().await; + assert!(batch.is_none()); + + Ok(()) + } + fn parquet_file_metrics() -> ParquetFileMetrics { let metrics = Arc::new(ExecutionPlanMetricsSet::new()); ParquetFileMetrics::new(0, "file.parquet", &metrics) diff --git a/datafusion/src/physical_plan/filter.rs b/datafusion/src/physical_plan/filter.rs index bab9f1e974d8..fe0f10313451 100644 --- a/datafusion/src/physical_plan/filter.rs +++ b/datafusion/src/physical_plan/filter.rs @@ -222,10 +222,9 @@ impl RecordBatchStream for FilterExecStream { mod tests { use super::*; - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::expressions::*; - use crate::physical_plan::file_format::CsvExec; + use crate::physical_plan::file_format::{CsvExec, PhysicalPlanConfig}; use crate::physical_plan::ExecutionPlan; use crate::scalar::ScalarValue; use crate::test::{self}; diff --git a/datafusion/src/physical_plan/limit.rs b/datafusion/src/physical_plan/limit.rs index 62df87987f50..f9c392a9056a 100644 --- a/datafusion/src/physical_plan/limit.rs +++ b/datafusion/src/physical_plan/limit.rs @@ -384,11 +384,10 @@ mod tests { use common::collect; use super::*; - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::common; - use crate::physical_plan::file_format::CsvExec; + use crate::physical_plan::file_format::{CsvExec, PhysicalPlanConfig}; use crate::test; #[tokio::test] diff --git a/datafusion/src/physical_plan/projection.rs b/datafusion/src/physical_plan/projection.rs index 4cc53196eb6a..eb335c2100ac 100644 --- a/datafusion/src/physical_plan/projection.rs +++ b/datafusion/src/physical_plan/projection.rs @@ -259,10 +259,9 @@ impl RecordBatchStream for ProjectionStream { mod tests { use super::*; - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::expressions::{self, col}; - use crate::physical_plan::file_format::CsvExec; + use crate::physical_plan::file_format::{CsvExec, PhysicalPlanConfig}; use crate::scalar::ScalarValue; use crate::test::{self}; use futures::future; diff --git a/datafusion/src/physical_plan/sort.rs b/datafusion/src/physical_plan/sort.rs index fffebd6008d6..a606906e8680 100644 --- a/datafusion/src/physical_plan/sort.rs +++ b/datafusion/src/physical_plan/sort.rs @@ -310,12 +310,14 @@ impl RecordBatchStream for SortStream { #[cfg(test)] mod tests { use super::*; - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::expressions::col; use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::{collect, file_format::CsvExec}; + use crate::physical_plan::{ + collect, + file_format::{CsvExec, PhysicalPlanConfig}, + }; use crate::test::assert_is_pending; use crate::test::exec::assert_strong_count_converges_to_zero; use crate::test::{self, exec::BlockingExec}; diff --git a/datafusion/src/physical_plan/sort_preserving_merge.rs b/datafusion/src/physical_plan/sort_preserving_merge.rs index ee0285c37780..62f4b941f7f7 100644 --- a/datafusion/src/physical_plan/sort_preserving_merge.rs +++ b/datafusion/src/physical_plan/sort_preserving_merge.rs @@ -658,7 +658,6 @@ impl RecordBatchStream for SortPreservingMergeStream { #[cfg(test)] mod tests { - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::metrics::MetricValue; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; @@ -668,7 +667,7 @@ mod tests { use crate::assert_batches_eq; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::expressions::col; - use crate::physical_plan::file_format::CsvExec; + use crate::physical_plan::file_format::{CsvExec, PhysicalPlanConfig}; use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::sort::SortExec; use crate::physical_plan::{collect, common}; diff --git a/datafusion/src/physical_plan/union.rs b/datafusion/src/physical_plan/union.rs index 01cb457351a3..418be630bed9 100644 --- a/datafusion/src/physical_plan/union.rs +++ b/datafusion/src/physical_plan/union.rs @@ -218,11 +218,14 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { #[cfg(test)] mod tests { use super::*; - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::{local::LocalFileSystem, ObjectStore}; use crate::test; + use crate::{ - physical_plan::{collect, file_format::CsvExec}, + physical_plan::{ + collect, + file_format::{CsvExec, PhysicalPlanConfig}, + }, scalar::ScalarValue, }; use arrow::record_batch::RecordBatch; diff --git a/datafusion/src/physical_plan/windows/mod.rs b/datafusion/src/physical_plan/windows/mod.rs index 6ff4a49fa29d..28bf40293612 100644 --- a/datafusion/src/physical_plan/windows/mod.rs +++ b/datafusion/src/physical_plan/windows/mod.rs @@ -175,11 +175,10 @@ pub(crate) fn find_ranges_in_range<'a>( #[cfg(test)] mod tests { use super::*; - use crate::datasource::file_format::PhysicalPlanConfig; use crate::datasource::object_store::local::LocalFileSystem; use crate::physical_plan::aggregates::AggregateFunction; use crate::physical_plan::expressions::col; - use crate::physical_plan::file_format::CsvExec; + use crate::physical_plan::file_format::{CsvExec, PhysicalPlanConfig}; use crate::physical_plan::{collect, Statistics}; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, aggr_test_schema, assert_is_pending}; From 32c5bf68bad7bf078b9c9968d014b67c3a441bde Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 09/17] [feat] specify supports_filter_pushdown --- datafusion/src/datasource/listing/helpers.rs | 234 +++++++++++-------- datafusion/src/datasource/listing/table.rs | 10 +- datafusion/src/execution/context.rs | 47 ++++ datafusion/src/test/object_store.rs | 42 +++- 4 files changed, 220 insertions(+), 113 deletions(-) diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs index 88d658ade92c..01f8427e2323 100644 --- a/datafusion/src/datasource/listing/helpers.rs +++ b/datafusion/src/datasource/listing/helpers.rs @@ -51,6 +51,91 @@ const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_"; const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_"; const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_"; +/// Check whether the given expression can be resolved using only the columns `col_names` +pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { + match expr { + // leaf + Expr::Literal(_) => true, + // TODO how to handle qualified / unqualified names? + Expr::Column(logical_plan::Column { ref name, .. }) => col_names.contains(name), + // unary + Expr::Alias(child, _) + | Expr::Not(child) + | Expr::IsNotNull(child) + | Expr::IsNull(child) + | Expr::Negative(child) + | Expr::Cast { expr: child, .. } + | Expr::TryCast { expr: child, .. } => expr_applicable_for_cols(col_names, child), + // binary + Expr::BinaryExpr { + ref left, + ref right, + .. + } => { + expr_applicable_for_cols(col_names, left) + && expr_applicable_for_cols(col_names, right) + } + // ternary + Expr::Between { + expr: item, + low, + high, + .. + } => { + expr_applicable_for_cols(col_names, item) + && expr_applicable_for_cols(col_names, low) + && expr_applicable_for_cols(col_names, high) + } + // variadic + Expr::ScalarFunction { fun, args } => match fun.volatility() { + Volatility::Immutable => args + .iter() + .all(|arg| expr_applicable_for_cols(col_names, arg)), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable => false, + Volatility::Volatile => false, + }, + Expr::ScalarUDF { fun, args } => match fun.signature.volatility { + Volatility::Immutable => args + .iter() + .all(|arg| expr_applicable_for_cols(col_names, arg)), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable => false, + Volatility::Volatile => false, + }, + Expr::InList { + expr: item, list, .. + } => { + expr_applicable_for_cols(col_names, item) + && list.iter().all(|e| expr_applicable_for_cols(col_names, e)) + } + Expr::Case { + expr, + when_then_expr, + else_expr, + } => { + let expr_constant = expr + .as_ref() + .map(|e| expr_applicable_for_cols(col_names, e)) + .unwrap_or(true); + let else_constant = else_expr + .as_ref() + .map(|e| expr_applicable_for_cols(col_names, e)) + .unwrap_or(true); + let when_then_constant = when_then_expr.iter().all(|(w, th)| { + expr_applicable_for_cols(col_names, w) + && expr_applicable_for_cols(col_names, th) + }); + expr_constant && else_constant && when_then_constant + } + // TODO other expressions are not handled yet: + // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases + // - Can `Wildcard` be considered as a `Literal`? + // - ScalarVariable could be `applicable`, but that would require access to the context + _ => false, + } +} + /// Partition the list of files into `n` groups pub fn split_files( partitioned_files: Vec, @@ -68,36 +153,72 @@ pub fn split_files( /// Discover the partitions on the given path and prune out files /// that belong to irrelevant partitions using `filters` expressions. +/// Assumes that `filters` only contains expressions that can be resolved +/// using partitioning columns only. +/// /// TODO for tables with many files (10k+), it will usually more efficient /// to first list the folders relative to the first partition dimension, /// prune those, then list only the contain of the remaining folders. pub async fn pruned_partition_list( store: &dyn ObjectStore, table_path: &str, - filters: &[Expr], + applicable_filters: &[Expr], file_extension: &str, table_partition_cols: &[String], ) -> Result { - if table_partition_cols.is_empty() || filters.is_empty() { + // if no partition col => simply list all the files + // if partition but no filter => parse the partition values while listing all the files + // otherwise => parse the partition values and serde them as a RecordBatch to filter them + if table_partition_cols.is_empty() { Ok(Box::pin( store .list_file_with_suffix(table_path, file_extension) .await? .map(|f| { Ok(PartitionedFile { - file_meta: f?, partition_values: vec![], + file_meta: f?, }) }), )) - } else { - let applicable_filters = filters - .iter() - .filter(|f| expr_applicable_for_cols(table_partition_cols, f)); - - let table_partition_cols = table_partition_cols.to_vec(); + } else if applicable_filters.is_empty() { let stream_path = table_path.to_owned(); + let table_partition_cols_stream = table_partition_cols.to_vec(); + Ok(Box::pin( + store + .list_file_with_suffix(table_path, file_extension) + .await? + .filter_map(move |f| { + let stream_path = stream_path.clone(); + let table_partition_cols_stream = table_partition_cols_stream.clone(); + async move { + let file_meta = match f { + Ok(fm) => fm, + Err(err) => return Some(Err(err)), + }; + let parsed_path = parse_partitions_for_path( + &stream_path, + file_meta.path(), + &table_partition_cols_stream, + ) + .map(|p| { + p.iter() + .map(|&pn| ScalarValue::Utf8(Some(pn.to_owned()))) + .collect() + }); + + parsed_path.map(|partition_values| { + Ok(PartitionedFile { + partition_values, + file_meta, + }) + }) + } + }), + )) + } else { // TODO avoid collecting but have a streaming memory table instead + let stream_path = table_path.to_owned(); let batches: Vec = store .list_file_with_suffix(table_path, file_extension) .await? @@ -107,9 +228,7 @@ pub async fn pruned_partition_list( // 1000 items at a time so batches of 1000 would be ideal with S3 as store. .chunks(1024) .map(|v| v.into_iter().collect::>>()) - .map(move |metas| { - paths_to_batch(&table_partition_cols, &stream_path, &metas?) - }) + .map(move |metas| paths_to_batch(table_partition_cols, &stream_path, &metas?)) .try_collect() .await?; @@ -259,91 +378,6 @@ fn parse_partitions_for_path<'a>( Some(part_values) } -/// Check whether the given expression can be resolved using only the columns `col_names` -fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { - match expr { - // leaf - Expr::Literal(_) => true, - // TODO how to handle qualified / unqualified names? - Expr::Column(logical_plan::Column { ref name, .. }) => col_names.contains(name), - // unary - Expr::Alias(child, _) - | Expr::Not(child) - | Expr::IsNotNull(child) - | Expr::IsNull(child) - | Expr::Negative(child) - | Expr::Cast { expr: child, .. } - | Expr::TryCast { expr: child, .. } => expr_applicable_for_cols(col_names, child), - // binary - Expr::BinaryExpr { - ref left, - ref right, - .. - } => { - expr_applicable_for_cols(col_names, left) - && expr_applicable_for_cols(col_names, right) - } - // ternary - Expr::Between { - expr: item, - low, - high, - .. - } => { - expr_applicable_for_cols(col_names, item) - && expr_applicable_for_cols(col_names, low) - && expr_applicable_for_cols(col_names, high) - } - // variadic - Expr::ScalarFunction { fun, args } => match fun.volatility() { - Volatility::Immutable => args - .iter() - .all(|arg| expr_applicable_for_cols(col_names, arg)), - // TODO: Stable functions could be `applicable`, but that would require access to the context - Volatility::Stable => false, - Volatility::Volatile => false, - }, - Expr::ScalarUDF { fun, args } => match fun.signature.volatility { - Volatility::Immutable => args - .iter() - .all(|arg| expr_applicable_for_cols(col_names, arg)), - // TODO: Stable functions could be `applicable`, but that would require access to the context - Volatility::Stable => false, - Volatility::Volatile => false, - }, - Expr::InList { - expr: item, list, .. - } => { - expr_applicable_for_cols(col_names, item) - && list.iter().all(|e| expr_applicable_for_cols(col_names, e)) - } - Expr::Case { - expr, - when_then_expr, - else_expr, - } => { - let expr_constant = expr - .as_ref() - .map(|e| expr_applicable_for_cols(col_names, e)) - .unwrap_or(true); - let else_constant = else_expr - .as_ref() - .map(|e| expr_applicable_for_cols(col_names, e)) - .unwrap_or(true); - let when_then_constant = when_then_expr.iter().all(|(w, th)| { - expr_applicable_for_cols(col_names, w) - && expr_applicable_for_cols(col_names, th) - }); - expr_constant && else_constant && when_then_constant - } - // TODO other expressions are not handled yet: - // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases - // - Can `Wildcard` be considered as a `Literal`? - // - ScalarVariable could be `applicable`, but that would require access to the context - _ => false, - } -} - #[cfg(test)] mod tests { use crate::{ @@ -462,11 +496,11 @@ mod tests { ("tablepath/part1=p1v1/file.parquet", 100), ("tablepath/part1=p1v2/part2=p2v1/file1.parquet", 100), ("tablepath/part1=p1v2/part2=p2v1/file2.parquet", 100), - ("tablepath/part1=p1v3/part2=p2v2/file2.parquet", 100), + ("tablepath/part1=p1v3/part2=p2v1/file2.parquet", 100), + ("tablepath/part1=p1v2/part2=p2v2/file2.parquet", 100), ]); let filter1 = Expr::eq(col("part1"), lit("p1v2")); - // filter2 cannot be resolved at partition pruning - let filter2 = Expr::eq(col("part2"), col("other")); + let filter2 = Expr::eq(col("part2"), lit("p2v1")); let pruned = pruned_partition_list( store.as_ref(), "tablepath/", diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index f49bd792da92..4bf78c57a35b 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -38,7 +38,7 @@ use crate::datasource::{ get_statistics_with_limit, object_store::ObjectStore, PartitionedFile, TableProvider, }; -use super::helpers::{pruned_partition_list, split_files}; +use super::helpers::{expr_applicable_for_cols, pruned_partition_list, split_files}; /// Options for creating a `ListingTable` pub struct ListingOptions { @@ -210,9 +210,13 @@ impl TableProvider for ListingTable { fn supports_filter_pushdown( &self, - _filter: &Expr, + filter: &Expr, ) -> Result { - Ok(TableProviderFilterPushDown::Inexact) + if expr_applicable_for_cols(&self.options.table_partition_cols, filter) { + Ok(TableProviderFilterPushDown::Exact) + } else { + Ok(TableProviderFilterPushDown::Unsupported) + } } } diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index 152d02f7308d..4ffea80cdf42 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -1152,6 +1152,8 @@ mod tests { use crate::physical_plan::functions::{make_scalar_function, Volatility}; use crate::physical_plan::{collect, collect_partitioned}; use crate::test; + use crate::test::object_store::TestObjectStore; + use crate::test_util::arrow_test_data; use crate::variable::VarType; use crate::{ assert_batches_eq, assert_batches_sorted_eq, @@ -3290,6 +3292,51 @@ mod tests { Ok(()) } + #[tokio::test] + async fn read_files_from_partitioned_path() -> Result<()> { + let mut ctx = ExecutionContext::new(); + + let testdata = arrow_test_data(); + let csv_file_path = format!("{}/csv/aggregate_test_100.csv", testdata); + let file_schema = aggr_test_schema(); + let object_store = TestObjectStore::new_mirror( + csv_file_path, + &[ + "mytable/date=2021-10-27/file.csv", + "mytable/date=2021-10-28/file.csv", + ], + ); + + let mut options = ListingOptions::new(Arc::new(CsvFormat::default())); + options.table_partition_cols = vec!["date".to_owned()]; + + let table = + ListingTable::new(object_store, "mytable".to_owned(), file_schema, options); + + ctx.register_table("t", Arc::new(table))?; + + let result = ctx + .sql("SELECT c1, date FROM t WHERE date='2021-10-27' LIMIT 5") + .await? + .collect() + .await?; + + let expected = vec![ + "+----+------------+", + "| c1 | date |", + "+----+------------+", + "| a | 2021-10-27 |", + "| b | 2021-10-27 |", + "| b | 2021-10-27 |", + "| c | 2021-10-27 |", + "| d | 2021-10-27 |", + "+----+------------+", + ]; + assert_batches_sorted_eq!(expected, &result); + + Ok(()) + } + #[tokio::test] async fn custom_query_planner() -> Result<()> { let mut ctx = ExecutionContext::with_config( diff --git a/datafusion/src/test/object_store.rs b/datafusion/src/test/object_store.rs index f82cfdbb8e99..9050e2215564 100644 --- a/datafusion/src/test/object_store.rs +++ b/datafusion/src/test/object_store.rs @@ -18,32 +18,47 @@ //! Object store implem used for testing use std::{ - io, + fs, io, io::{Cursor, Read}, sync::Arc, }; -use crate::{ - datasource::object_store::{ - FileMeta, FileMetaStream, ListEntryStream, ObjectReader, ObjectStore, SizedFile, - }, - error::{DataFusionError, Result}, +use crate::datasource::object_store::{ + local::LocalFileSystem, FileMeta, FileMetaStream, ListEntryStream, ObjectReader, + ObjectStore, SizedFile, }; +use crate::error::{DataFusionError, Result}; + use async_trait::async_trait; use futures::{stream, AsyncRead, StreamExt}; #[derive(Debug)] /// An object store implem that is useful for testing. -/// The `ObjectReader`s are filled with zero bytes. +/// Can either generate `ObjectReader`s that are filled with zero- +/// bytes or mirror a given file to multiple path. pub struct TestObjectStore { /// The `(path,size)` of the files that "exist" in the store - pub files: Vec<(String, u64)>, + files: Vec<(String, u64)>, + /// The file that will be read at all path. If none fille the + /// file with zero-bytes. + mirrored_file: Option, } impl TestObjectStore { pub fn new_arc(files: &[(&str, u64)]) -> Arc { Arc::new(Self { files: files.iter().map(|f| (f.0.to_owned(), f.1)).collect(), + mirrored_file: None, + }) + } + pub fn new_mirror(mirrored_file: String, paths: &[&str]) -> Arc { + let metadata = fs::metadata(&mirrored_file).expect("Local file metadata"); + Arc::new(Self { + files: paths + .iter() + .map(|&f| (f.to_owned(), metadata.len())) + .collect(), + mirrored_file: Some(mirrored_file), }) } } @@ -81,8 +96,15 @@ impl ObjectStore for TestObjectStore { fn file_reader(&self, file: SizedFile) -> Result> { match self.files.iter().find(|item| file.path == item.0) { - Some((_, size)) if *size == file.size => { - Ok(Arc::new(EmptyObjectReader(*size))) + Some(&(_, size)) if size == file.size => { + if let Some(mirrored_file) = &self.mirrored_file { + Ok(LocalFileSystem {}.file_reader(SizedFile { + path: mirrored_file.clone(), + size, + })?) + } else { + Ok(Arc::new(EmptyObjectReader(size))) + } } Some(_) => Err(DataFusionError::IoError(io::Error::new( io::ErrorKind::NotFound, From 10b52eb666ac33b29f55a147e8e827a4c1bb51f4 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 10/17] [fix] parquet rowgroup pruning not kicking in --- datafusion/src/datasource/listing/helpers.rs | 38 +++++++++++++------- datafusion/src/datasource/listing/table.rs | 5 ++- 2 files changed, 29 insertions(+), 14 deletions(-) diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs index 01f8427e2323..b37602c7ffeb 100644 --- a/datafusion/src/datasource/listing/helpers.rs +++ b/datafusion/src/datasource/listing/helpers.rs @@ -51,7 +51,11 @@ const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_"; const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_"; const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_"; -/// Check whether the given expression can be resolved using only the columns `col_names` +/// Check whether the given expression can be resolved using only the columns `col_names`. +/// This means that if this function returns true: +/// - the table provider can filter the table partition values with this expression +/// - the expression can be marked as `TableProviderFilterPushDown::Exact` once this filtering +/// was performed pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { match expr { // leaf @@ -153,8 +157,8 @@ pub fn split_files( /// Discover the partitions on the given path and prune out files /// that belong to irrelevant partitions using `filters` expressions. -/// Assumes that `filters` only contains expressions that can be resolved -/// using partitioning columns only. +/// `filters` might contain expressions that can be resolved only at the +/// file level (e.g. Parquet row group pruning). /// /// TODO for tables with many files (10k+), it will usually more efficient /// to first list the folders relative to the first partition dimension, @@ -162,15 +166,13 @@ pub fn split_files( pub async fn pruned_partition_list( store: &dyn ObjectStore, table_path: &str, - applicable_filters: &[Expr], + filters: &[Expr], file_extension: &str, table_partition_cols: &[String], ) -> Result { - // if no partition col => simply list all the files - // if partition but no filter => parse the partition values while listing all the files - // otherwise => parse the partition values and serde them as a RecordBatch to filter them + // if no partition col => simply list all the files if table_partition_cols.is_empty() { - Ok(Box::pin( + return Ok(Box::pin( store .list_file_with_suffix(table_path, file_extension) .await? @@ -180,9 +182,17 @@ pub async fn pruned_partition_list( file_meta: f?, }) }), - )) - } else if applicable_filters.is_empty() { - let stream_path = table_path.to_owned(); + )); + } + + let applicable_filters: Vec<_> = filters + .iter() + .filter(|f| expr_applicable_for_cols(table_partition_cols, f)) + .collect(); + let stream_path = table_path.to_owned(); + if applicable_filters.is_empty() { + // parse the partition values while listing all the files + // TODO we might avoid parsing the partition values if they are not used in any projection let table_partition_cols_stream = table_partition_cols.to_vec(); Ok(Box::pin( store @@ -217,8 +227,8 @@ pub async fn pruned_partition_list( }), )) } else { + // parse the partition values and serde them as a RecordBatch to filter them // TODO avoid collecting but have a streaming memory table instead - let stream_path = table_path.to_owned(); let batches: Vec = store .list_file_with_suffix(table_path, file_extension) .await? @@ -501,10 +511,12 @@ mod tests { ]); let filter1 = Expr::eq(col("part1"), lit("p1v2")); let filter2 = Expr::eq(col("part2"), lit("p2v1")); + // filter3 cannot be resolved at partition pruning + let filter3 = Expr::eq(col("part2"), col("other")); let pruned = pruned_partition_list( store.as_ref(), "tablepath/", - &[filter1, filter2], + &[filter1, filter2, filter3], ".parquet", &[String::from("part1"), String::from("part2")], ) diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index 4bf78c57a35b..64aadb04b000 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -213,9 +213,12 @@ impl TableProvider for ListingTable { filter: &Expr, ) -> Result { if expr_applicable_for_cols(&self.options.table_partition_cols, filter) { + // if filter can be handled by partiton pruning, it is exact Ok(TableProviderFilterPushDown::Exact) } else { - Ok(TableProviderFilterPushDown::Unsupported) + // otherwise, we still might be able to handle the filter with file + // level mechanisms such as Parquet row group pruning. + Ok(TableProviderFilterPushDown::Inexact) } } } From f207b9265929dc5a6c8215061d451d28d1c03009 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 10:49:35 +0200 Subject: [PATCH 11/17] [test] added proper integration tests --- datafusion/src/execution/context.rs | 47 ------ datafusion/src/test/object_store.rs | 41 ++--- datafusion/tests/common.rs | 40 +++++ datafusion/tests/path_partition.rs | 235 ++++++++++++++++++++++++++++ datafusion/tests/sql.rs | 22 +-- 5 files changed, 287 insertions(+), 98 deletions(-) create mode 100644 datafusion/tests/common.rs create mode 100644 datafusion/tests/path_partition.rs diff --git a/datafusion/src/execution/context.rs b/datafusion/src/execution/context.rs index 4ffea80cdf42..152d02f7308d 100644 --- a/datafusion/src/execution/context.rs +++ b/datafusion/src/execution/context.rs @@ -1152,8 +1152,6 @@ mod tests { use crate::physical_plan::functions::{make_scalar_function, Volatility}; use crate::physical_plan::{collect, collect_partitioned}; use crate::test; - use crate::test::object_store::TestObjectStore; - use crate::test_util::arrow_test_data; use crate::variable::VarType; use crate::{ assert_batches_eq, assert_batches_sorted_eq, @@ -3292,51 +3290,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn read_files_from_partitioned_path() -> Result<()> { - let mut ctx = ExecutionContext::new(); - - let testdata = arrow_test_data(); - let csv_file_path = format!("{}/csv/aggregate_test_100.csv", testdata); - let file_schema = aggr_test_schema(); - let object_store = TestObjectStore::new_mirror( - csv_file_path, - &[ - "mytable/date=2021-10-27/file.csv", - "mytable/date=2021-10-28/file.csv", - ], - ); - - let mut options = ListingOptions::new(Arc::new(CsvFormat::default())); - options.table_partition_cols = vec!["date".to_owned()]; - - let table = - ListingTable::new(object_store, "mytable".to_owned(), file_schema, options); - - ctx.register_table("t", Arc::new(table))?; - - let result = ctx - .sql("SELECT c1, date FROM t WHERE date='2021-10-27' LIMIT 5") - .await? - .collect() - .await?; - - let expected = vec![ - "+----+------------+", - "| c1 | date |", - "+----+------------+", - "| a | 2021-10-27 |", - "| b | 2021-10-27 |", - "| b | 2021-10-27 |", - "| c | 2021-10-27 |", - "| d | 2021-10-27 |", - "+----+------------+", - ]; - assert_batches_sorted_eq!(expected, &result); - - Ok(()) - } - #[tokio::test] async fn custom_query_planner() -> Result<()> { let mut ctx = ExecutionContext::with_config( diff --git a/datafusion/src/test/object_store.rs b/datafusion/src/test/object_store.rs index 9050e2215564..e93b4cd2d410 100644 --- a/datafusion/src/test/object_store.rs +++ b/datafusion/src/test/object_store.rs @@ -14,51 +14,35 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. - //! Object store implem used for testing use std::{ - fs, io, + io, io::{Cursor, Read}, sync::Arc, }; -use crate::datasource::object_store::{ - local::LocalFileSystem, FileMeta, FileMetaStream, ListEntryStream, ObjectReader, - ObjectStore, SizedFile, +use crate::{ + datasource::object_store::{ + FileMeta, FileMetaStream, ListEntryStream, ObjectReader, ObjectStore, SizedFile, + }, + error::{DataFusionError, Result}, }; -use crate::error::{DataFusionError, Result}; - use async_trait::async_trait; use futures::{stream, AsyncRead, StreamExt}; #[derive(Debug)] /// An object store implem that is useful for testing. -/// Can either generate `ObjectReader`s that are filled with zero- -/// bytes or mirror a given file to multiple path. +/// `ObjectReader`s are filled with zero bytes. pub struct TestObjectStore { /// The `(path,size)` of the files that "exist" in the store files: Vec<(String, u64)>, - /// The file that will be read at all path. If none fille the - /// file with zero-bytes. - mirrored_file: Option, } impl TestObjectStore { pub fn new_arc(files: &[(&str, u64)]) -> Arc { Arc::new(Self { files: files.iter().map(|f| (f.0.to_owned(), f.1)).collect(), - mirrored_file: None, - }) - } - pub fn new_mirror(mirrored_file: String, paths: &[&str]) -> Arc { - let metadata = fs::metadata(&mirrored_file).expect("Local file metadata"); - Arc::new(Self { - files: paths - .iter() - .map(|&f| (f.to_owned(), metadata.len())) - .collect(), - mirrored_file: Some(mirrored_file), }) } } @@ -96,15 +80,8 @@ impl ObjectStore for TestObjectStore { fn file_reader(&self, file: SizedFile) -> Result> { match self.files.iter().find(|item| file.path == item.0) { - Some(&(_, size)) if size == file.size => { - if let Some(mirrored_file) = &self.mirrored_file { - Ok(LocalFileSystem {}.file_reader(SizedFile { - path: mirrored_file.clone(), - size, - })?) - } else { - Ok(Arc::new(EmptyObjectReader(size))) - } + Some((_, size)) if *size == file.size => { + Ok(Arc::new(EmptyObjectReader(*size))) } Some(_) => Err(DataFusionError::IoError(io::Error::new( io::ErrorKind::NotFound, diff --git a/datafusion/tests/common.rs b/datafusion/tests/common.rs new file mode 100644 index 000000000000..3490db5e091f --- /dev/null +++ b/datafusion/tests/common.rs @@ -0,0 +1,40 @@ +// 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. + +//! methods that are common to multiple integration test setups + +use std::sync::Arc; + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + +pub fn aggr_test_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("c1", DataType::Utf8, false), + Field::new("c2", DataType::UInt32, false), + Field::new("c3", DataType::Int8, false), + Field::new("c4", DataType::Int16, false), + Field::new("c5", DataType::Int32, false), + Field::new("c6", DataType::Int64, false), + Field::new("c7", DataType::UInt8, false), + Field::new("c8", DataType::UInt16, false), + Field::new("c9", DataType::UInt32, false), + Field::new("c10", DataType::UInt64, false), + Field::new("c11", DataType::Float32, false), + Field::new("c12", DataType::Float64, false), + Field::new("c13", DataType::Utf8, false), + ])) +} diff --git a/datafusion/tests/path_partition.rs b/datafusion/tests/path_partition.rs new file mode 100644 index 000000000000..897baaa154df --- /dev/null +++ b/datafusion/tests/path_partition.rs @@ -0,0 +1,235 @@ +// 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. + +//! Test queries on partitioned datasets + +use std::{fs, io, sync::Arc}; + +use async_trait::async_trait; +use datafusion::{ + assert_batches_sorted_eq, + datasource::{ + file_format::csv::CsvFormat, + listing::{ListingOptions, ListingTable}, + object_store::{ + local::LocalFileSystem, FileMeta, FileMetaStream, ListEntryStream, + ObjectReader, ObjectStore, SizedFile, + }, + }, + error::{DataFusionError, Result}, + prelude::ExecutionContext, + test_util::arrow_test_data, +}; +use futures::{stream, StreamExt}; + +mod common; + +#[tokio::test] +async fn csv_filter_with_file_col() -> Result<()> { + let mut ctx = ExecutionContext::new(); + + register_partitioned_aggregate_csv( + &mut ctx, + &[ + "mytable/date=2021-10-27/file.csv", + "mytable/date=2021-10-28/file.csv", + ], + &["date"], + "mytable", + ); + + let result = ctx + .sql("SELECT c1, c2 FROM t WHERE date='2021-10-27' and date!=c1 LIMIT 5") + .await? + .collect() + .await?; + + let expected = vec![ + "+----+----+", + "| c1 | c2 |", + "+----+----+", + "| a | 1 |", + "| b | 1 |", + "| b | 5 |", + "| c | 2 |", + "| d | 5 |", + "+----+----+", + ]; + assert_batches_sorted_eq!(expected, &result); + + Ok(()) +} + +#[tokio::test] +async fn csv_projection_on_partition() -> Result<()> { + let mut ctx = ExecutionContext::new(); + + register_partitioned_aggregate_csv( + &mut ctx, + &[ + "mytable/date=2021-10-27/file.csv", + "mytable/date=2021-10-28/file.csv", + ], + &["date"], + "mytable", + ); + + let result = ctx + .sql("SELECT c1, date FROM t WHERE date='2021-10-27' LIMIT 5") + .await? + .collect() + .await?; + + let expected = vec![ + "+----+------------+", + "| c1 | date |", + "+----+------------+", + "| a | 2021-10-27 |", + "| b | 2021-10-27 |", + "| b | 2021-10-27 |", + "| c | 2021-10-27 |", + "| d | 2021-10-27 |", + "+----+------------+", + ]; + assert_batches_sorted_eq!(expected, &result); + + Ok(()) +} + +#[tokio::test] +async fn csv_grouping_by_partition() -> Result<()> { + let mut ctx = ExecutionContext::new(); + + register_partitioned_aggregate_csv( + &mut ctx, + &[ + "mytable/date=2021-10-26/file.csv", + "mytable/date=2021-10-27/file.csv", + "mytable/date=2021-10-28/file.csv", + ], + &["date"], + "mytable", + ); + + let result = ctx + .sql("SELECT date, count(*), count(distinct(c1)) FROM t WHERE date<='2021-10-27' GROUP BY date") + .await? + .collect() + .await?; + + let expected = vec![ + "+------------+-----------------+----------------------+", + "| date | COUNT(UInt8(1)) | COUNT(DISTINCT t.c1) |", + "+------------+-----------------+----------------------+", + "| 2021-10-26 | 100 | 5 |", + "| 2021-10-27 | 100 | 5 |", + "+------------+-----------------+----------------------+", + ]; + assert_batches_sorted_eq!(expected, &result); + + Ok(()) +} + +fn register_partitioned_aggregate_csv( + ctx: &mut ExecutionContext, + store_paths: &[&str], + partition_cols: &[&str], + table_path: &str, +) { + let testdata = arrow_test_data(); + let csv_file_path = format!("{}/csv/aggregate_test_100.csv", testdata); + let file_schema = common::aggr_test_schema(); + let object_store = MirroringObjectStore::new_arc(csv_file_path, store_paths); + + let mut options = ListingOptions::new(Arc::new(CsvFormat::default())); + options.table_partition_cols = partition_cols.iter().map(|&s| s.to_owned()).collect(); + + let table = + ListingTable::new(object_store, table_path.to_owned(), file_schema, options); + + ctx.register_table("t", Arc::new(table)) + .expect("registering listing table failed"); +} + +#[derive(Debug)] +/// An object store implem that is mirrors a given file to multiple paths. +pub struct MirroringObjectStore { + /// The `(path,size)` of the files that "exist" in the store + files: Vec, + /// The file that will be read at all path + mirrored_file: String, + /// Size of the mirrored file + file_size: u64, +} + +impl MirroringObjectStore { + pub fn new_arc(mirrored_file: String, paths: &[&str]) -> Arc { + let metadata = fs::metadata(&mirrored_file).expect("Local file metadata"); + Arc::new(Self { + files: paths.iter().map(|&f| f.to_owned()).collect(), + mirrored_file, + file_size: metadata.len(), + }) + } +} + +#[async_trait] +impl ObjectStore for MirroringObjectStore { + async fn list_file(&self, prefix: &str) -> Result { + let prefix = prefix.to_owned(); + let size = self.file_size; + Ok(Box::pin( + stream::iter( + self.files + .clone() + .into_iter() + .filter(move |f| f.starts_with(&prefix)), + ) + .map(move |f| { + Ok(FileMeta { + sized_file: SizedFile { path: f, size }, + last_modified: None, + }) + }), + )) + } + + async fn list_dir( + &self, + _prefix: &str, + _delimiter: Option, + ) -> Result { + unimplemented!() + } + + fn file_reader(&self, file: SizedFile) -> Result> { + assert_eq!( + self.file_size, file.size, + "Requested files should have the same size as the mirrored file" + ); + match self.files.iter().find(|&item| &file.path == item) { + Some(_) => Ok(LocalFileSystem {}.file_reader(SizedFile { + path: self.mirrored_file.clone(), + size: self.file_size, + })?), + None => Err(DataFusionError::IoError(io::Error::new( + io::ErrorKind::NotFound, + "not in provided test list", + ))), + } + } +} diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs index f3dba3fc2ad1..3403bde60480 100644 --- a/datafusion/tests/sql.rs +++ b/datafusion/tests/sql.rs @@ -49,6 +49,8 @@ use datafusion::{ }; use datafusion::{execution::context::ExecutionContext, physical_plan::displayable}; +mod common; + #[tokio::test] async fn nyc() -> Result<()> { // schema for nyxtaxi csv files @@ -3195,24 +3197,6 @@ async fn explain_analyze_runs_optimizers() { assert_contains!(actual, expected); } -fn aggr_test_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Utf8, false), - Field::new("c2", DataType::UInt32, false), - Field::new("c3", DataType::Int8, false), - Field::new("c4", DataType::Int16, false), - Field::new("c5", DataType::Int32, false), - Field::new("c6", DataType::Int64, false), - Field::new("c7", DataType::UInt8, false), - Field::new("c8", DataType::UInt16, false), - Field::new("c9", DataType::UInt32, false), - Field::new("c10", DataType::UInt64, false), - Field::new("c11", DataType::Float32, false), - Field::new("c12", DataType::Float64, false), - Field::new("c13", DataType::Utf8, false), - ])) -} - async fn register_aggregate_csv_by_sql(ctx: &mut ExecutionContext) { let testdata = datafusion::test_util::arrow_test_data(); @@ -3256,7 +3240,7 @@ async fn register_aggregate_csv_by_sql(ctx: &mut ExecutionContext) { async fn register_aggregate_csv(ctx: &mut ExecutionContext) -> Result<()> { let testdata = datafusion::test_util::arrow_test_data(); - let schema = aggr_test_schema(); + let schema = common::aggr_test_schema(); ctx.register_csv( "aggregate_test_100", &format!("{}/csv/aggregate_test_100.csv", testdata), From dfa6d0e79985a82d71ad373b991cc4c0cb5acd50 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 13:31:12 +0200 Subject: [PATCH 12/17] [test] check output schemas --- datafusion/src/datasource/listing/table.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index 64aadb04b000..81eaed5488a7 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -278,7 +278,7 @@ mod tests { object_store::local::LocalFileSystem, }, logical_plan::{col, lit}, - test::object_store::TestObjectStore, + test::{columns, object_store::TestObjectStore}, }; use super::*; @@ -314,9 +314,14 @@ mod tests { collect_stat: true, }; - let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); + let file_schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]); - let table = ListingTable::new(store, "table/".to_owned(), Arc::new(schema), opt); + let table = + ListingTable::new(store, "table/".to_owned(), Arc::new(file_schema), opt); + 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")); @@ -327,6 +332,10 @@ mod tests { .expect("Empty execution plan"); assert!(scan.as_any().is::()); + assert_eq!( + columns(&scan.schema()), + vec!["a".to_owned(), "p1".to_owned()] + ); Ok(()) } From f8edb38aeac5c934e079874942308bab1c62d7a7 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 15:08:02 +0200 Subject: [PATCH 13/17] [test] add integ tests for partitioned parquet --- datafusion/tests/path_partition.rs | 97 +++++++++++++++++++++++++++++- 1 file changed, 95 insertions(+), 2 deletions(-) diff --git a/datafusion/tests/path_partition.rs b/datafusion/tests/path_partition.rs index 897baaa154df..74179c4f3f1e 100644 --- a/datafusion/tests/path_partition.rs +++ b/datafusion/tests/path_partition.rs @@ -23,7 +23,7 @@ use async_trait::async_trait; use datafusion::{ assert_batches_sorted_eq, datasource::{ - file_format::csv::CsvFormat, + file_format::{csv::CsvFormat, parquet::ParquetFormat}, listing::{ListingOptions, ListingTable}, object_store::{ local::LocalFileSystem, FileMeta, FileMetaStream, ListEntryStream, @@ -32,7 +32,7 @@ use datafusion::{ }, error::{DataFusionError, Result}, prelude::ExecutionContext, - test_util::arrow_test_data, + test_util::{arrow_test_data, parquet_test_data}, }; use futures::{stream, StreamExt}; @@ -144,6 +144,73 @@ async fn csv_grouping_by_partition() -> Result<()> { Ok(()) } +#[tokio::test] +async fn parquet_multiple_partitions() -> Result<()> { + let mut ctx = ExecutionContext::new(); + + register_partitioned_alltypes_parquet( + &mut ctx, + &[ + "year=2021/month=09/day=09/file.parquet", + "year=2021/month=10/day=09/file.parquet", + "year=2021/month=10/day=28/file.parquet", + ], + &["year", "month", "day"], + "", + ) + .await; + + let result = ctx + .sql("SELECT id, day FROM t WHERE day=month ORDER BY id") + .await? + .collect() + .await?; + + let expected = vec![ + "+----+-----+", + "| id | day |", + "+----+-----+", + "| 0 | 09 |", + "| 1 | 09 |", + "| 2 | 09 |", + "| 3 | 09 |", + "| 4 | 09 |", + "| 5 | 09 |", + "| 6 | 09 |", + "| 7 | 09 |", + "+----+-----+", + ]; + assert_batches_sorted_eq!(expected, &result); + + Ok(()) +} + +#[tokio::test] +async fn parquet_overlapping_columns() -> Result<()> { + let mut ctx = ExecutionContext::new(); + + // `id` is both a column of the file and a partitioning col + register_partitioned_alltypes_parquet( + &mut ctx, + &[ + "id=1/file.parquet", + "id=2/file.parquet", + "id=3/file.parquet", + ], + &["id"], + "", + ) + .await; + + let result = ctx.sql("SELECT id FROM t WHERE id=1 ORDER BY id").await; + + assert!( + result.is_err(), + "Dupplicate qualified name should raise error" + ); + Ok(()) +} + fn register_partitioned_aggregate_csv( ctx: &mut ExecutionContext, store_paths: &[&str], @@ -165,6 +232,32 @@ fn register_partitioned_aggregate_csv( .expect("registering listing table failed"); } +async fn register_partitioned_alltypes_parquet( + ctx: &mut ExecutionContext, + store_paths: &[&str], + partition_cols: &[&str], + table_path: &str, +) { + let testdata = parquet_test_data(); + let parquet_file_path = format!("{}/alltypes_plain.parquet", testdata); + let object_store = + MirroringObjectStore::new_arc(parquet_file_path.clone(), store_paths); + + let mut options = ListingOptions::new(Arc::new(ParquetFormat::default())); + options.table_partition_cols = partition_cols.iter().map(|&s| s.to_owned()).collect(); + + let file_schema = options + .infer_schema(Arc::clone(&object_store), store_paths[0]) + .await + .expect("Parquet schema inference failed"); + + let table = + ListingTable::new(object_store, table_path.to_owned(), file_schema, options); + + ctx.register_table("t", Arc::new(table)) + .expect("registering listing table failed"); +} + #[derive(Debug)] /// An object store implem that is mirrors a given file to multiple paths. pub struct MirroringObjectStore { From c5cfcfb8d103ff62b606e821ff719c3d5248d9c9 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 28 Oct 2021 17:45:11 +0200 Subject: [PATCH 14/17] [fix] wrong schema passed to get_statistics_with_limit --- datafusion/src/datasource/listing/table.rs | 5 +- datafusion/src/datasource/mod.rs | 15 ++--- .../src/physical_plan/file_format/mod.rs | 2 +- datafusion/tests/path_partition.rs | 66 ++++++++++++++++++- 4 files changed, 76 insertions(+), 12 deletions(-) diff --git a/datafusion/src/datasource/listing/table.rs b/datafusion/src/datasource/listing/table.rs index 81eaed5488a7..aadc340b46c9 100644 --- a/datafusion/src/datasource/listing/table.rs +++ b/datafusion/src/datasource/listing/table.rs @@ -224,8 +224,9 @@ impl TableProvider for ListingTable { } impl ListingTable { - /// Get the list of files for a scan. The list is grouped to let the execution plan - /// know how the files should be distributed to different threads / executors. + /// 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, filters: &'a [Expr], diff --git a/datafusion/src/datasource/mod.rs b/datafusion/src/datasource/mod.rs index efea47b1d437..9f4f77f7ea28 100644 --- a/datafusion/src/datasource/mod.rs +++ b/datafusion/src/datasource/mod.rs @@ -37,22 +37,21 @@ use crate::scalar::ScalarValue; use futures::StreamExt; use std::pin::Pin; -/// Get all files as well as the summary statistic -/// if the optional `limit` is provided, includes only sufficient files -/// needed to read up to `limit` number of rows +/// Get all files as well as the file level summary statistics (no statistic for partition columns). +/// If the optional `limit` is provided, includes only sufficient files. +/// Needed to read up to `limit` number of rows. /// TODO fix case where `num_rows` and `total_byte_size` are not defined (stat should be None instead of Some(0)) -/// TODO check that stats for partition columns are correct pub async fn get_statistics_with_limit( all_files: impl Stream>, - schema: SchemaRef, + file_schema: SchemaRef, limit: Option, ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; let mut total_byte_size = 0; - let mut null_counts = vec![0; schema.fields().len()]; + let mut null_counts = vec![0; file_schema.fields().len()]; let mut has_statistics = false; - let (mut max_values, mut min_values) = create_max_min_accs(&schema); + let (mut max_values, mut min_values) = create_max_min_accs(&file_schema); let mut num_rows = 0; let mut is_exact = true; @@ -105,7 +104,7 @@ pub async fn get_statistics_with_limit( let column_stats = if has_statistics { Some(get_col_stats( - &*schema, + &*file_schema, null_counts, &mut max_values, &mut min_values, diff --git a/datafusion/src/physical_plan/file_format/mod.rs b/datafusion/src/physical_plan/file_format/mod.rs index 64788b9bc046..cd30c6f7e05b 100644 --- a/datafusion/src/physical_plan/file_format/mod.rs +++ b/datafusion/src/physical_plan/file_format/mod.rs @@ -108,7 +108,7 @@ impl PhysicalPlanConfig { DEFAULT_PARTITION_COLUMN_DATATYPE.clone(), false, )); - // TODO provide accurate stat for partition column + // TODO provide accurate stat for partition column (#1186) table_cols_stats.push(ColumnStatistics::default()) } } diff --git a/datafusion/tests/path_partition.rs b/datafusion/tests/path_partition.rs index 74179c4f3f1e..789511065fc8 100644 --- a/datafusion/tests/path_partition.rs +++ b/datafusion/tests/path_partition.rs @@ -31,6 +31,7 @@ use datafusion::{ }, }, error::{DataFusionError, Result}, + physical_plan::ColumnStatistics, prelude::ExecutionContext, test_util::{arrow_test_data, parquet_test_data}, }; @@ -157,6 +158,7 @@ async fn parquet_multiple_partitions() -> Result<()> { ], &["year", "month", "day"], "", + "alltypes_plain.parquet", ) .await; @@ -185,6 +187,65 @@ async fn parquet_multiple_partitions() -> Result<()> { Ok(()) } +#[tokio::test] +async fn parquet_statistics() -> Result<()> { + let mut ctx = ExecutionContext::new(); + + register_partitioned_alltypes_parquet( + &mut ctx, + &[ + "year=2021/month=09/day=09/file.parquet", + "year=2021/month=10/day=09/file.parquet", + "year=2021/month=10/day=28/file.parquet", + ], + &["year", "month", "day"], + "", + // This is the only file we found in the test set with + // actual stats. It has 1 column / 1 row. + "single_nan.parquet", + ) + .await; + + //// NO PROJECTION //// + let logical_plan = ctx.sql("SELECT * FROM t").await?.to_logical_plan(); + + let physical_plan = ctx.create_physical_plan(&logical_plan).await?; + assert_eq!(physical_plan.schema().fields().len(), 4); + + let stat_cols = physical_plan + .statistics() + .column_statistics + .expect("col stats should be defined"); + assert_eq!(stat_cols.len(), 4); + // stats for the first col are read from the parquet file + assert_eq!(stat_cols[0].null_count, Some(3)); + // TODO assert partition column (1,2,3) stats once implemented (#1186) + assert_eq!(stat_cols[1], ColumnStatistics::default()); + assert_eq!(stat_cols[2], ColumnStatistics::default()); + assert_eq!(stat_cols[3], ColumnStatistics::default()); + + //// WITH PROJECTION //// + let logical_plan = ctx + .sql("SELECT mycol, day FROM t WHERE day='28'") + .await? + .to_logical_plan(); + + let physical_plan = ctx.create_physical_plan(&logical_plan).await?; + assert_eq!(physical_plan.schema().fields().len(), 2); + + let stat_cols = physical_plan + .statistics() + .column_statistics + .expect("col stats should be defined"); + assert_eq!(stat_cols.len(), 2); + // stats for the first col are read from the parquet file + assert_eq!(stat_cols[0].null_count, Some(1)); + // TODO assert partition column stats once implemented (#1186) + assert_eq!(stat_cols[1], ColumnStatistics::default()); + + Ok(()) +} + #[tokio::test] async fn parquet_overlapping_columns() -> Result<()> { let mut ctx = ExecutionContext::new(); @@ -199,6 +260,7 @@ async fn parquet_overlapping_columns() -> Result<()> { ], &["id"], "", + "alltypes_plain.parquet", ) .await; @@ -237,14 +299,16 @@ async fn register_partitioned_alltypes_parquet( store_paths: &[&str], partition_cols: &[&str], table_path: &str, + source_file: &str, ) { let testdata = parquet_test_data(); - let parquet_file_path = format!("{}/alltypes_plain.parquet", testdata); + let parquet_file_path = format!("{}/{}", testdata, source_file); let object_store = MirroringObjectStore::new_arc(parquet_file_path.clone(), store_paths); let mut options = ListingOptions::new(Arc::new(ParquetFormat::default())); options.table_partition_cols = partition_cols.iter().map(|&s| s.to_owned()).collect(); + options.collect_stat = true; let file_schema = options .infer_schema(Arc::clone(&object_store), store_paths[0]) From cb0789e5f789fd80d5238d4fd5f22b682c309b01 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Fri, 29 Oct 2021 10:59:22 +0200 Subject: [PATCH 15/17] [fix] improvements following review https://github.com/apache/arrow-datafusion/pull/1141#pullrequestreview-792371776 --- datafusion/src/datasource/listing/helpers.rs | 160 +++++++++--------- .../src/physical_plan/file_format/mod.rs | 70 +++++++- 2 files changed, 143 insertions(+), 87 deletions(-) diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs index b37602c7ffeb..59a4333806e0 100644 --- a/datafusion/src/datasource/listing/helpers.rs +++ b/datafusion/src/datasource/listing/helpers.rs @@ -37,7 +37,7 @@ use log::debug; use crate::{ error::Result, execution::context::ExecutionContext, - logical_plan::{self, Expr}, + logical_plan::{self, Expr, ExpressionVisitor, Recursion}, physical_plan::functions::Volatility, scalar::ScalarValue, }; @@ -51,93 +51,83 @@ const FILE_SIZE_COLUMN_NAME: &str = "_df_part_file_size_"; const FILE_PATH_COLUMN_NAME: &str = "_df_part_file_path_"; const FILE_MODIFIED_COLUMN_NAME: &str = "_df_part_file_modified_"; +/// The `ExpressionVisitor` for `expr_applicable_for_cols`. Walks the tree to +/// validate that the given expression is applicable with only the `col_names` +/// set of columns. +struct ApplicabilityVisitor<'a> { + col_names: &'a [String], + is_applicable: &'a mut bool, +} + +impl ApplicabilityVisitor<'_> { + fn visit_volatility(self, volatility: Volatility) -> Recursion { + match volatility { + Volatility::Immutable => Recursion::Continue(self), + // TODO: Stable functions could be `applicable`, but that would require access to the context + Volatility::Stable | Volatility::Volatile => { + *self.is_applicable = false; + Recursion::Stop(self) + } + } + } +} + +impl ExpressionVisitor for ApplicabilityVisitor<'_> { + fn pre_visit(self, expr: &Expr) -> Result> { + let rec = match expr { + Expr::Column(logical_plan::Column { ref name, .. }) => { + *self.is_applicable &= self.col_names.contains(name); + Recursion::Stop(self) // leaf node anyway + } + Expr::Literal(_) + | Expr::Alias(_, _) + | Expr::ScalarVariable(_) + | Expr::Not(_) + | Expr::IsNotNull(_) + | Expr::IsNull(_) + | Expr::Negative(_) + | Expr::Cast { .. } + | Expr::TryCast { .. } + | Expr::BinaryExpr { .. } + | Expr::Between { .. } + | Expr::InList { .. } + | Expr::Case { .. } => Recursion::Continue(self), + + Expr::ScalarFunction { fun, .. } => self.visit_volatility(fun.volatility()), + Expr::ScalarUDF { fun, .. } => { + self.visit_volatility(fun.signature.volatility) + } + + // TODO other expressions are not handled yet: + // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases + // - Can `Wildcard` be considered as a `Literal`? + // - ScalarVariable could be `applicable`, but that would require access to the context + Expr::AggregateUDF { .. } + | Expr::AggregateFunction { .. } + | Expr::Sort { .. } + | Expr::WindowFunction { .. } + | Expr::Wildcard => { + *self.is_applicable = false; + Recursion::Stop(self) + } + }; + Ok(rec) + } +} + /// Check whether the given expression can be resolved using only the columns `col_names`. /// This means that if this function returns true: /// - the table provider can filter the table partition values with this expression /// - the expression can be marked as `TableProviderFilterPushDown::Exact` once this filtering /// was performed pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { - match expr { - // leaf - Expr::Literal(_) => true, - // TODO how to handle qualified / unqualified names? - Expr::Column(logical_plan::Column { ref name, .. }) => col_names.contains(name), - // unary - Expr::Alias(child, _) - | Expr::Not(child) - | Expr::IsNotNull(child) - | Expr::IsNull(child) - | Expr::Negative(child) - | Expr::Cast { expr: child, .. } - | Expr::TryCast { expr: child, .. } => expr_applicable_for_cols(col_names, child), - // binary - Expr::BinaryExpr { - ref left, - ref right, - .. - } => { - expr_applicable_for_cols(col_names, left) - && expr_applicable_for_cols(col_names, right) - } - // ternary - Expr::Between { - expr: item, - low, - high, - .. - } => { - expr_applicable_for_cols(col_names, item) - && expr_applicable_for_cols(col_names, low) - && expr_applicable_for_cols(col_names, high) - } - // variadic - Expr::ScalarFunction { fun, args } => match fun.volatility() { - Volatility::Immutable => args - .iter() - .all(|arg| expr_applicable_for_cols(col_names, arg)), - // TODO: Stable functions could be `applicable`, but that would require access to the context - Volatility::Stable => false, - Volatility::Volatile => false, - }, - Expr::ScalarUDF { fun, args } => match fun.signature.volatility { - Volatility::Immutable => args - .iter() - .all(|arg| expr_applicable_for_cols(col_names, arg)), - // TODO: Stable functions could be `applicable`, but that would require access to the context - Volatility::Stable => false, - Volatility::Volatile => false, - }, - Expr::InList { - expr: item, list, .. - } => { - expr_applicable_for_cols(col_names, item) - && list.iter().all(|e| expr_applicable_for_cols(col_names, e)) - } - Expr::Case { - expr, - when_then_expr, - else_expr, - } => { - let expr_constant = expr - .as_ref() - .map(|e| expr_applicable_for_cols(col_names, e)) - .unwrap_or(true); - let else_constant = else_expr - .as_ref() - .map(|e| expr_applicable_for_cols(col_names, e)) - .unwrap_or(true); - let when_then_constant = when_then_expr.iter().all(|(w, th)| { - expr_applicable_for_cols(col_names, w) - && expr_applicable_for_cols(col_names, th) - }); - expr_constant && else_constant && when_then_constant - } - // TODO other expressions are not handled yet: - // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases - // - Can `Wildcard` be considered as a `Literal`? - // - ScalarVariable could be `applicable`, but that would require access to the context - _ => false, - } + let mut is_applicable = true; + expr.accept(ApplicabilityVisitor { + col_names, + is_applicable: &mut is_applicable, + }) + .unwrap(); + is_applicable } /// Partition the list of files into `n` groups @@ -191,8 +181,10 @@ pub async fn pruned_partition_list( .collect(); let stream_path = table_path.to_owned(); if applicable_filters.is_empty() { - // parse the partition values while listing all the files - // TODO we might avoid parsing the partition values if they are not used in any projection + // Parse the partition values while listing all the files + // Note: We might avoid parsing the partition values if they are not used in any projection, + // but the cost of parsing will likely be far dominated by the time to fetch the listing from + // the object store. let table_partition_cols_stream = table_partition_cols.to_vec(); Ok(Box::pin( store diff --git a/datafusion/src/physical_plan/file_format/mod.rs b/datafusion/src/physical_plan/file_format/mod.rs index cd30c6f7e05b..d460e9830fe5 100644 --- a/datafusion/src/physical_plan/file_format/mod.rs +++ b/datafusion/src/physical_plan/file_format/mod.rs @@ -166,7 +166,12 @@ impl<'a> Display for FileGroupsDisplay<'a> { } } -/// A helper that projects partition columns into the file record batches +/// A helper that projects partition columns into the file record batches. +/// +/// One interesting trick is the usage of a cache for the key buffers of the partition column +/// dictionaries. Indeed, the partition columns are constant, so the dictionaries that represent them +/// have all their keys equal to 0. This enables us to re-use the same "all-zero" buffer across batches, +/// which makes the space consumption of the partition columns O(batch_size) instead of O(record_count). struct PartitionColumnProjector { /// An Arrow buffer initialized to zeros that represents the key array of all partition /// columns (partition columns are materialized by dictionary arrays with only one @@ -202,7 +207,7 @@ impl PartitionColumnProjector { } } - // Transform the batch read from the fileby inserting the partitioning columns + // Transform the batch read from the file by inserting the partitioning columns // to the right positions as deduced from `projected_schema` // - file_batch: batch read from the file, with internal projection applied // - partition_values: the list of partition values, one for each partition column @@ -379,6 +384,8 @@ mod tests { let (proj_schema, _) = conf.project(); // created a projector for that projected schema let mut proj = PartitionColumnProjector::new(proj_schema, &partition_cols); + + // project first batch let projected_batch = proj .project( // file_batch is ok here because we kept all the file cols in the projection @@ -390,7 +397,6 @@ mod tests { ], ) .expect("Projection of partition columns into record batch failed"); - let expected = vec![ "+---+----+----+------+-----+", "| a | b | c | year | day |", @@ -401,6 +407,64 @@ mod tests { "+---+----+----+------+-----+", ]; crate::assert_batches_eq!(expected, &[projected_batch]); + + // project another batch that is larger than the previous one + let file_batch = build_table_i32( + ("a", &vec![5, 6, 7, 8, 9]), + ("b", &vec![-10, -9, -8, -7, -6]), + ("c", &vec![12, 13, 14, 15, 16]), + ); + let projected_batch = proj + .project( + // file_batch is ok here because we kept all the file cols in the projection + file_batch, + &[ + ScalarValue::Utf8(Some("2021".to_owned())), + ScalarValue::Utf8(Some("10".to_owned())), + ScalarValue::Utf8(Some("27".to_owned())), + ], + ) + .expect("Projection of partition columns into record batch failed"); + let expected = vec![ + "+---+-----+----+------+-----+", + "| a | b | c | year | day |", + "+---+-----+----+------+-----+", + "| 5 | -10 | 12 | 2021 | 27 |", + "| 6 | -9 | 13 | 2021 | 27 |", + "| 7 | -8 | 14 | 2021 | 27 |", + "| 8 | -7 | 15 | 2021 | 27 |", + "| 9 | -6 | 16 | 2021 | 27 |", + "+---+-----+----+------+-----+", + ]; + crate::assert_batches_eq!(expected, &[projected_batch]); + + // project another batch that is smaller than the previous one + let file_batch = build_table_i32( + ("a", &vec![0, 1, 3]), + ("b", &vec![2, 3, 4]), + ("c", &vec![4, 5, 6]), + ); + let projected_batch = proj + .project( + // file_batch is ok here because we kept all the file cols in the projection + file_batch, + &[ + ScalarValue::Utf8(Some("2021".to_owned())), + ScalarValue::Utf8(Some("10".to_owned())), + ScalarValue::Utf8(Some("28".to_owned())), + ], + ) + .expect("Projection of partition columns into record batch failed"); + let expected = vec![ + "+---+---+---+------+-----+", + "| a | b | c | year | day |", + "+---+---+---+------+-----+", + "| 0 | 2 | 4 | 2021 | 28 |", + "| 1 | 3 | 5 | 2021 | 28 |", + "| 3 | 4 | 6 | 2021 | 28 |", + "+---+---+---+------+-----+", + ]; + crate::assert_batches_eq!(expected, &[projected_batch]); } // sets default for configs that play no role in projections From 5d34be6df60fb2c36129d51b41ffce80c749ee67 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 1 Nov 2021 09:46:34 -0400 Subject: [PATCH 16/17] Add GetIndexedField to the list of exprs that can be in predicates --- datafusion/src/datasource/listing/helpers.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs index 59a4333806e0..608834c325c0 100644 --- a/datafusion/src/datasource/listing/helpers.rs +++ b/datafusion/src/datasource/listing/helpers.rs @@ -91,6 +91,7 @@ impl ExpressionVisitor for ApplicabilityVisitor<'_> { | Expr::BinaryExpr { .. } | Expr::Between { .. } | Expr::InList { .. } + | Expr::GetIndexedField { .. } | Expr::Case { .. } => Recursion::Continue(self), Expr::ScalarFunction { fun, .. } => self.visit_volatility(fun.volatility()), From a4ec80132e05d1e263409eadd1a4071f26392c84 Mon Sep 17 00:00:00 2001 From: rdettai Date: Mon, 1 Nov 2021 16:03:50 +0100 Subject: [PATCH 17/17] Update datafusion/src/datasource/listing/helpers.rs Co-authored-by: Jiayu Liu --- datafusion/src/datasource/listing/helpers.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/src/datasource/listing/helpers.rs b/datafusion/src/datasource/listing/helpers.rs index 608834c325c0..912179c36f06 100644 --- a/datafusion/src/datasource/listing/helpers.rs +++ b/datafusion/src/datasource/listing/helpers.rs @@ -139,6 +139,7 @@ pub fn split_files( if partitioned_files.is_empty() { return vec![]; } + // effectively this is div with rounding up instead of truncating let chunk_size = (partitioned_files.len() + n - 1) / n; partitioned_files .chunks(chunk_size)