diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index 63a91ca9aa1c..b5bcb8243ea9 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -16,29 +16,31 @@ // under the License. //! Functions that are query-able and searchable via the `\h` command + +use std::fmt; +use std::fs::File; +use std::str::FromStr; +use std::sync::Arc; + use arrow::array::{Int64Array, StringArray}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; -use async_trait::async_trait; - use datafusion::catalog::{Session, TableFunctionImpl}; use datafusion::common::{plan_err, Column}; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::logical_expr::Expr; -use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::memory::MemorySourceConfig; use datafusion::physical_plan::ExecutionPlan; use datafusion::scalar::ScalarValue; + +use async_trait::async_trait; use parquet::basic::ConvertedType; use parquet::data_type::{ByteArray, FixedLenByteArray}; use parquet::file::reader::FileReader; use parquet::file::serialized_reader::SerializedFileReader; use parquet::file::statistics::Statistics; -use std::fmt; -use std::fs::File; -use std::str::FromStr; -use std::sync::Arc; #[derive(Debug)] pub enum Function { @@ -240,11 +242,11 @@ impl TableProvider for ParquetMetadataTable { _filters: &[Expr], _limit: Option, ) -> Result> { - Ok(Arc::new(MemoryExec::try_new( + Ok(MemorySourceConfig::try_new_exec( &[vec![self.batch.clone()]], TableProvider::schema(self), projection.cloned(), - )?)) + )?) } } diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 7e2139370e20..43dc592b997e 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -25,18 +25,14 @@ use std::sync::Arc; use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; use arrow::datatypes::SchemaRef; -use async_trait::async_trait; -use bytes::Bytes; use datafusion::catalog::Session; use datafusion::common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::parquet::{ - ParquetAccessPlan, ParquetExecBuilder, -}; +use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; use datafusion::datasource::physical_plan::{ - parquet::ParquetFileReaderFactory, FileMeta, FileScanConfig, + parquet::ParquetFileReaderFactory, FileMeta, FileScanConfig, ParquetSource, }; use datafusion::datasource::TableProvider; use datafusion::execution::object_store::ObjectStoreUrl; @@ -56,6 +52,9 @@ use datafusion::physical_optimizer::pruning::PruningPredicate; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; + +use async_trait::async_trait; +use bytes::Bytes; use futures::future::BoxFuture; use futures::FutureExt; use object_store::ObjectStore; @@ -83,8 +82,8 @@ use url::Url; /// Specifically, this example illustrates how to: /// 1. Use [`ParquetFileReaderFactory`] to avoid re-reading parquet metadata on each query /// 2. Use [`PruningPredicate`] for predicate analysis -/// 3. Pass a row group selection to [`ParquetExec`] -/// 4. Pass a row selection (within a row group) to [`ParquetExec`] +/// 3. Pass a row group selection to [`ParquetSource`] +/// 4. Pass a row selection (within a row group) to [`ParquetSource`] /// /// Note this is a *VERY* low level example for people who want to build their /// own custom indexes (e.g. for low latency queries). Most users should use @@ -94,38 +93,38 @@ use url::Url; /// /// # Diagram /// -/// This diagram shows how the `ParquetExec` is configured to do only a single +/// This diagram shows how the `DataSourceExec` with `ParquetSource` is configured to do only a single /// (range) read from a parquet file, for the data that is needed. It does /// not read the file footer or any of the row groups that are not needed. /// /// ```text /// ┌───────────────────────┐ The TableProvider configures the -/// │ ┌───────────────────┐ │ ParquetExec: +/// │ ┌───────────────────┐ │ DataSourceExec: /// │ │ │ │ /// │ └───────────────────┘ │ /// │ ┌───────────────────┐ │ /// Row │ │ │ │ 1. To read only specific Row -/// Groups │ └───────────────────┘ │ Groups (the ParquetExec tries +/// Groups │ └───────────────────┘ │ Groups (the DataSourceExec tries /// │ ┌───────────────────┐ │ to reduce this further based /// │ │ │ │ on metadata) -/// │ └───────────────────┘ │ ┌────────────────────┐ -/// │ ┌───────────────────┐ │ │ │ -/// │ │ │◀┼ ─ ─ ┐ │ ParquetExec │ -/// │ └───────────────────┘ │ │ (Parquet Reader) │ -/// │ ... │ └ ─ ─ ─ ─│ │ -/// │ ┌───────────────────┐ │ │ ╔═══════════════╗ │ -/// │ │ │ │ │ ║ParquetMetadata║ │ -/// │ └───────────────────┘ │ │ ╚═══════════════╝ │ -/// │ ╔═══════════════════╗ │ └────────────────────┘ +/// │ └───────────────────┘ │ ┌──────────────────────┐ +/// │ ┌───────────────────┐ │ │ │ +/// │ │ │◀┼ ─ ─ ┐ │ DataSourceExec │ +/// │ └───────────────────┘ │ │ │ (Parquet Reader) │ +/// │ ... │ └ ─ ─ ─ ─│ │ +/// │ ┌───────────────────┐ │ │ ╔═══════════════╗ │ +/// │ │ │ │ │ ║ParquetMetadata║ │ +/// │ └───────────────────┘ │ │ ╚═══════════════╝ │ +/// │ ╔═══════════════════╗ │ └──────────────────────┘ /// │ ║ Thrift metadata ║ │ /// │ ╚═══════════════════╝ │ 1. With cached ParquetMetadata, so -/// └───────────────────────┘ the ParquetExec does not re-read / +/// └───────────────────────┘ the ParquetSource does not re-read / /// Parquet File decode the thrift footer /// /// ``` /// /// Within a Row Group, Column Chunks store data in DataPages. This example also -/// shows how to configure the ParquetExec to read a `RowSelection` (row ranges) +/// shows how to configure the ParquetSource to read a `RowSelection` (row ranges) /// which will skip unneeded data pages. This requires that the Parquet file has /// a [Page Index]. /// @@ -135,15 +134,15 @@ use url::Url; /// │ │ Data Page is not fetched or decoded. /// │ ┌───────────────────┐ │ Note this requires a PageIndex /// │ │ ┌──────────┐ │ │ -/// Row │ │ │DataPage 0│ │ │ ┌────────────────────┐ -/// Groups │ │ └──────────┘ │ │ │ │ -/// │ │ ┌──────────┐ │ │ │ ParquetExec │ -/// │ │ ... │DataPage 1│ ◀┼ ┼ ─ ─ ─ │ (Parquet Reader) │ -/// │ │ └──────────┘ │ │ └ ─ ─ ─ ─ ─│ │ -/// │ │ ┌──────────┐ │ │ │ ╔═══════════════╗ │ -/// │ │ │DataPage 2│ │ │ If only rows │ ║ParquetMetadata║ │ -/// │ │ └──────────┘ │ │ from DataPage 1 │ ╚═══════════════╝ │ -/// │ └───────────────────┘ │ are selected, └────────────────────┘ +/// Row │ │ │DataPage 0│ │ │ ┌──────────────────────┐ +/// Groups │ │ └──────────┘ │ │ │ │ +/// │ │ ┌──────────┐ │ │ │ DataSourceExec │ +/// │ │ ... │DataPage 1│ ◀┼ ┼ ─ ─ ─ │ (Parquet Reader) │ +/// │ │ └──────────┘ │ │ └ ─ ─ ─ ─ ─│ │ +/// │ │ ┌──────────┐ │ │ │ ╔═══════════════╗ │ +/// │ │ │DataPage 2│ │ │ If only rows │ ║ParquetMetadata║ │ +/// │ │ └──────────┘ │ │ from DataPage 1 │ ╚═══════════════╝ │ +/// │ └───────────────────┘ │ are selected, └──────────────────────┘ /// │ │ only DataPage 1 /// │ ... │ is fetched and /// │ │ decoded @@ -211,7 +210,7 @@ async fn main() -> Result<()> { // pages that must be decoded // // Note: in order to prune pages, the Page Index must be loaded and the - // ParquetExec will load it on demand if not present. To avoid a second IO + // DataSourceExec will load it on demand if not present. To avoid a second IO // during query, this example loaded the Page Index preemptively by setting // `ArrowReader::with_page_index` in `IndexedFile::try_new` provider.set_use_row_selection(true); @@ -478,31 +477,34 @@ impl TableProvider for IndexTableProvider { let partitioned_file = indexed_file .partitioned_file() - // provide the starting access plan to the ParquetExec by + // provide the starting access plan to the DataSourceExec by // storing it as "extensions" on PartitionedFile .with_extensions(Arc::new(access_plan) as _); // Prepare for scanning let schema = self.schema(); let object_store_url = ObjectStoreUrl::parse("file://")?; - let file_scan_config = FileScanConfig::new(object_store_url, schema) - .with_limit(limit) - .with_projection(projection.cloned()) - .with_file(partitioned_file); // Configure a factory interface to avoid re-reading the metadata for each file let reader_factory = CachedParquetFileReaderFactory::new(Arc::clone(&self.object_store)) .with_file(indexed_file); - // Finally, put it all together into a ParquetExec - Ok(ParquetExecBuilder::new(file_scan_config) - // provide the predicate so the ParquetExec can try and prune - // row groups internally - .with_predicate(predicate) - // provide the factory to create parquet reader without re-reading metadata - .with_parquet_file_reader_factory(Arc::new(reader_factory)) - .build_arc()) + let file_source = Arc::new( + ParquetSource::default() + // provide the predicate so the DataSourceExec can try and prune + // row groups internally + .with_predicate(Arc::clone(&schema), predicate) + // provide the factory to create parquet reader without re-reading metadata + .with_parquet_file_reader_factory(Arc::new(reader_factory)), + ); + let file_scan_config = FileScanConfig::new(object_store_url, schema, file_source) + .with_limit(limit) + .with_projection(projection.cloned()) + .with_file(partitioned_file); + + // Finally, put it all together into a DataSourceExec + Ok(file_scan_config.new_exec()) } /// Tell DataFusion to push filters down to the scan method diff --git a/datafusion-examples/examples/csv_json_opener.rs b/datafusion-examples/examples/csv_json_opener.rs index 0b71432f621c..7613578e8c3a 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -18,18 +18,21 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema}; +use datafusion::datasource::physical_plan::JsonSource; use datafusion::{ assert_batches_eq, datasource::{ + data_source::FileSource, file_format::file_compression_type::FileCompressionType, listing::PartitionedFile, object_store::ObjectStoreUrl, - physical_plan::{CsvConfig, CsvOpener, FileScanConfig, FileStream, JsonOpener}, + physical_plan::{CsvSource, FileScanConfig, FileStream, JsonOpener}, }, error::Result, physical_plan::metrics::ExecutionPlanMetricsSet, test_util::aggr_test_schema, }; + use futures::StreamExt; use object_store::{local::LocalFileSystem, memory::InMemory, ObjectStore}; @@ -48,29 +51,27 @@ async fn csv_opener() -> Result<()> { let object_store = Arc::new(LocalFileSystem::new()); let schema = aggr_test_schema(); - let config = CsvConfig::new( - 8192, - schema.clone(), - Some(vec![12, 0]), - true, - b',', - b'"', - None, - object_store, - Some(b'#'), - ); - - let opener = CsvOpener::new(Arc::new(config), FileCompressionType::UNCOMPRESSED); - let testdata = datafusion::test_util::arrow_test_data(); let path = format!("{testdata}/csv/aggregate_test_100.csv"); let path = std::path::Path::new(&path).canonicalize()?; - let scan_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema) - .with_projection(Some(vec![12, 0])) - .with_limit(Some(5)) - .with_file(PartitionedFile::new(path.display().to_string(), 10)); + let scan_config = FileScanConfig::new( + ObjectStoreUrl::local_filesystem(), + Arc::clone(&schema), + Arc::new(CsvSource::default()), + ) + .with_projection(Some(vec![12, 0])) + .with_limit(Some(5)) + .with_file(PartitionedFile::new(path.display().to_string(), 10)); + + let config = CsvSource::new(true, b',', b'"') + .with_comment(Some(b'#')) + .with_schema(schema) + .with_batch_size(8192) + .with_projection(&scan_config); + + let opener = config.create_file_opener(Ok(object_store), &scan_config, 0)?; let mut result = vec![]; let mut stream = @@ -120,13 +121,21 @@ async fn json_opener() -> Result<()> { Arc::new(object_store), ); - let scan_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema) - .with_projection(Some(vec![1, 0])) - .with_limit(Some(5)) - .with_file(PartitionedFile::new(path.to_string(), 10)); - - let mut stream = - FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new())?; + let scan_config = FileScanConfig::new( + ObjectStoreUrl::local_filesystem(), + schema, + Arc::new(JsonSource::default()), + ) + .with_projection(Some(vec![1, 0])) + .with_limit(Some(5)) + .with_file(PartitionedFile::new(path.to_string(), 10)); + + let mut stream = FileStream::new( + &scan_config, + 0, + Arc::new(opener), + &ExecutionPlanMetricsSet::new(), + )?; let mut result = vec![]; while let Some(batch) = stream.next().await.transpose()? { result.push(batch); diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index 5493fa21968e..4d85ce882923 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -22,6 +22,7 @@ use arrow::{ datatypes::{DataType, Field, Schema, SchemaRef, UInt64Type}, }; use datafusion::common::{GetExt, Statistics}; +use datafusion::datasource::data_source::FileSource; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::physical_expr::LexRequirement; use datafusion::physical_expr::PhysicalExpr; @@ -126,6 +127,10 @@ impl FileFormat for TSVFileFormat { .create_writer_physical_plan(input, state, conf, order_requirements) .await } + + fn file_source(&self) -> Arc { + self.csv_file_format.file_source() + } } #[derive(Default, Debug)] diff --git a/datafusion-examples/examples/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index 2f1b6333373e..20809a1121c1 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -19,10 +19,11 @@ use std::sync::Arc; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ListingOptions, PartitionedFile}; -use datafusion::datasource::physical_plan::ParquetExec; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::error::DataFusionError; use datafusion::execution::context::SessionContext; use datafusion::physical_plan::metrics::MetricValue; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::{ execute_stream, visit_execution_plan, ExecutionPlan, ExecutionPlanVisitor, }; @@ -96,15 +97,24 @@ impl ExecutionPlanVisitor for ParquetExecVisitor { /// or `post_visit` (visit each node after its children/inputs) fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { // If needed match on a specific `ExecutionPlan` node type - let maybe_parquet_exec = plan.as_any().downcast_ref::(); - if let Some(parquet_exec) = maybe_parquet_exec { - self.file_groups = Some(parquet_exec.base_config().file_groups.clone()); + if let Some(data_source) = plan.as_any().downcast_ref::() { + let source = data_source.source(); + if let Some(file_config) = source.as_any().downcast_ref::() { + if file_config + .file_source() + .as_any() + .downcast_ref::() + .is_some() + { + self.file_groups = Some(file_config.file_groups.clone()); - let metrics = match parquet_exec.metrics() { - None => return Ok(true), - Some(metrics) => metrics, - }; - self.bytes_scanned = metrics.sum_by_name("bytes_scanned"); + let metrics = match data_source.metrics() { + None => return Ok(true), + Some(metrics) => metrics, + }; + self.bytes_scanned = metrics.sum_by_name("bytes_scanned"); + } + } } Ok(true) } diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index 67fa038e2d31..f465699abed2 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -27,7 +27,7 @@ use datafusion::common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::datasource::TableProvider; use datafusion::execution::object_store::ObjectStoreUrl; use datafusion::logical_expr::{ @@ -241,11 +241,14 @@ impl TableProvider for IndexTableProvider { let files = self.index.get_files(predicate.clone())?; let object_store_url = ObjectStoreUrl::parse("file://")?; - let mut file_scan_config = FileScanConfig::new(object_store_url, self.schema()) - .with_projection(projection.cloned()) - .with_limit(limit); - - // Transform to the format needed to pass to ParquetExec + let source = + Arc::new(ParquetSource::default().with_predicate(self.schema(), predicate)); + let mut file_scan_config = + FileScanConfig::new(object_store_url, self.schema(), source) + .with_projection(projection.cloned()) + .with_limit(limit); + + // Transform to the format needed to pass to DataSourceExec // Create one file group per file (default to scanning them all in parallel) for (file_name, file_size) in files { let path = self.dir.join(file_name); @@ -255,11 +258,7 @@ impl TableProvider for IndexTableProvider { file_size, )); } - let exec = ParquetExec::builder(file_scan_config) - .with_predicate(predicate) - .build_arc(); - - Ok(exec) + Ok(file_scan_config.new_exec()) } /// Tell DataFusion to push filters down to the scan method diff --git a/datafusion-examples/examples/remote_catalog.rs b/datafusion-examples/examples/remote_catalog.rs index e44b0c9569f9..f84c6a0302ce 100644 --- a/datafusion-examples/examples/remote_catalog.rs +++ b/datafusion-examples/examples/remote_catalog.rs @@ -38,7 +38,7 @@ use datafusion::common::Result; use datafusion::common::{assert_batches_eq, internal_datafusion_err, plan_err}; use datafusion::execution::SendableRecordBatchStream; use datafusion::logical_expr::{Expr, TableType}; -use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::memory::MemorySourceConfig; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{DataFrame, SessionContext}; @@ -254,10 +254,11 @@ impl TableProvider for RemoteTable { .await? .try_collect() .await?; - Ok(Arc::new(MemoryExec::try_new( + let exec = MemorySourceConfig::try_new_exec( &[batches], self.schema.clone(), projection.cloned(), - )?)) + )?; + Ok(exec) } } diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index 75c7645c18a5..afba4c390f71 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -29,7 +29,7 @@ use datafusion::execution::context::ExecutionProps; use datafusion::logical_expr::simplify::SimplifyContext; use datafusion::logical_expr::{Expr, TableType}; use datafusion::optimizer::simplify_expressions::ExprSimplifier; -use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::memory::MemorySourceConfig; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; use std::fs::File; @@ -120,11 +120,11 @@ impl TableProvider for LocalCsvTable { } else { self.batches.clone() }; - Ok(Arc::new(MemoryExec::try_new( + Ok(MemorySourceConfig::try_new_exec( &[batches], TableProvider::schema(self), projection.cloned(), - )?)) + )?) } } diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index 7d87a37b3b9c..53c245ecc2b5 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -33,7 +33,7 @@ use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMerge use datafusion::physical_plan::{ collect, expressions::{col, PhysicalSortExpr}, - memory::MemoryExec, + memory::MemorySourceConfig, }; use datafusion::prelude::SessionContext; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -55,13 +55,13 @@ fn sort_preserving_merge_operator( }) .collect::(); - let exec = MemoryExec::try_new( + let exec = MemorySourceConfig::try_new_exec( &batches.into_iter().map(|rb| vec![rb]).collect::>(), schema, None, ) .unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let task_ctx = session_ctx.task_ctx(); let rt = Runtime::new().unwrap(); rt.block_on(collect(merge, task_ctx)).unwrap(); diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 14e80ce364e3..4d71d4c56a6d 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -79,17 +79,17 @@ use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::{ execution::context::TaskContext, physical_plan::{ - coalesce_partitions::CoalescePartitionsExec, memory::MemoryExec, + coalesce_partitions::CoalescePartitionsExec, memory::MemorySourceConfig, sorts::sort_preserving_merge::SortPreservingMergeExec, ExecutionPlan, ExecutionPlanProperties, }, prelude::SessionContext, }; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// Benchmarks for SortPreservingMerge stream use criterion::{criterion_group, criterion_main, Criterion}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::StreamExt; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; @@ -167,8 +167,8 @@ impl BenchCase { let schema = partitions[0][0].schema(); let sort = make_sort_exprs(schema.as_ref()); - let exec = MemoryExec::try_new(partitions, schema, None).unwrap(); - let plan = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); + let plan = Arc::new(SortPreservingMergeExec::new(sort, exec)); Self { runtime, @@ -186,9 +186,8 @@ impl BenchCase { let schema = partitions[0][0].schema(); let sort = make_sort_exprs(schema.as_ref()); - let exec = MemoryExec::try_new(partitions, schema, None).unwrap(); - let exec = - SortExec::new(sort.clone(), Arc::new(exec)).with_preserve_partitioning(true); + let source = MemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); + let exec = SortExec::new(sort.clone(), source).with_preserve_partitioning(true); let plan = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); Self { @@ -208,8 +207,8 @@ impl BenchCase { let schema = partitions[0][0].schema(); let sort = make_sort_exprs(schema.as_ref()); - let exec = MemoryExec::try_new(partitions, schema, None).unwrap(); - let exec = Arc::new(CoalescePartitionsExec::new(Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); + let exec = Arc::new(CoalescePartitionsExec::new(exec)); let plan = Arc::new(SortExec::new(sort, exec)); Self { @@ -229,8 +228,8 @@ impl BenchCase { let schema = partitions[0][0].schema(); let sort = make_sort_exprs(schema.as_ref()); - let exec = MemoryExec::try_new(partitions, schema, None).unwrap(); - let exec = SortExec::new(sort, Arc::new(exec)).with_preserve_partitioning(true); + let source = MemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); + let exec = SortExec::new(sort, source).with_preserve_partitioning(true); let plan = Arc::new(CoalescePartitionsExec::new(Arc::new(exec))); Self { diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs new file mode 100644 index 000000000000..3984ffa64c76 --- /dev/null +++ b/datafusion/core/src/datasource/data_source.rs @@ -0,0 +1,65 @@ +// 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. + +//! DataSource and FileSource trait implementations + +use std::any::Any; +use std::fmt; +use std::fmt::Formatter; +use std::sync::Arc; + +use crate::datasource::physical_plan::{FileOpener, FileScanConfig}; + +use arrow_schema::SchemaRef; +use datafusion_common::Statistics; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_physical_plan::DisplayFormatType; + +use object_store::ObjectStore; + +/// Common behaviors that every file format needs to implement. +/// +/// See initialization examples on `ParquetSource`, `CsvSource` +pub trait FileSource: Send + Sync { + /// Creates a `dyn FileOpener` based on given parameters + fn create_file_opener( + &self, + object_store: datafusion_common::Result>, + base_config: &FileScanConfig, + partition: usize, + ) -> datafusion_common::Result>; + /// Any + fn as_any(&self) -> &dyn Any; + /// Initialize new type with batch size configuration + fn with_batch_size(&self, batch_size: usize) -> Arc; + /// Initialize new instance with a new schema + fn with_schema(&self, schema: SchemaRef) -> Arc; + /// Initialize new instance with projection information + fn with_projection(&self, config: &FileScanConfig) -> Arc; + /// Initialize new instance with projected statistics + fn with_statistics(&self, statistics: Statistics) -> Arc; + /// Return execution plan metrics + fn metrics(&self) -> &ExecutionPlanMetricsSet; + /// Return projected statistics + fn statistics(&self) -> datafusion_common::Result; + /// String representation of file source such as "csv", "json", "parquet" + fn file_type(&self) -> &str; + /// Format FileType specific information + fn fmt_extra(&self, _t: DisplayFormatType, _f: &mut Formatter) -> fmt::Result { + Ok(()) + } +} diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 1d9827ae0ab5..e88144c2ea16 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -32,7 +32,7 @@ use super::FileFormatFactory; use crate::datasource::file_format::write::get_writer_schema; use crate::datasource::file_format::FileFormat; use crate::datasource::physical_plan::{ - ArrowExec, FileGroupDisplay, FileScanConfig, FileSink, FileSinkConfig, + ArrowSource, FileGroupDisplay, FileScanConfig, FileSink, FileSinkConfig, }; use crate::error::Result; use crate::execution::context::SessionState; @@ -51,11 +51,12 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; +use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::Bytes; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::BoxStream; use futures::StreamExt; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; @@ -169,11 +170,11 @@ impl FileFormat for ArrowFormat { async fn create_physical_plan( &self, _state: &SessionState, - conf: FileScanConfig, + mut conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { - let exec = ArrowExec::new(conf); - Ok(Arc::new(exec)) + conf = conf.with_source(Arc::new(ArrowSource::default())); + Ok(conf.new_exec()) } async fn create_writer_physical_plan( @@ -191,6 +192,10 @@ impl FileFormat for ArrowFormat { Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) } + + fn file_source(&self) -> Arc { + Arc::new(ArrowSource::default()) + } } /// Implements [`FileSink`] for writing to arrow_ipc files diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index f854b9506a64..8e5192d12f5d 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -22,6 +22,17 @@ use std::collections::HashMap; use std::fmt; use std::sync::Arc; +use super::file_compression_type::FileCompressionType; +use super::FileFormat; +use super::FileFormatFactory; +use crate::datasource::avro_to_arrow::read_avro_schema_from_reader; +use crate::datasource::physical_plan::{AvroSource, FileScanConfig}; +use crate::error::Result; +use crate::execution::context::SessionState; +use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::Statistics; + +use crate::datasource::data_source::FileSource; use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; use async_trait::async_trait; @@ -32,16 +43,6 @@ use datafusion_common::DEFAULT_AVRO_EXTENSION; use datafusion_physical_expr::PhysicalExpr; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; -use super::file_compression_type::FileCompressionType; -use super::FileFormat; -use super::FileFormatFactory; -use crate::datasource::avro_to_arrow::read_avro_schema_from_reader; -use crate::datasource::physical_plan::{AvroExec, FileScanConfig}; -use crate::error::Result; -use crate::execution::context::SessionState; -use crate::physical_plan::ExecutionPlan; -use crate::physical_plan::Statistics; - #[derive(Default)] /// Factory struct used to create [AvroFormat] pub struct AvroFormatFactory; @@ -147,11 +148,15 @@ impl FileFormat for AvroFormat { async fn create_physical_plan( &self, _state: &SessionState, - conf: FileScanConfig, + mut conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { - let exec = AvroExec::new(conf); - Ok(Arc::new(exec)) + conf = conf.with_source(self.file_source()); + Ok(conf.new_exec()) + } + + fn file_source(&self) -> Arc { + Arc::new(AvroSource::new()) } } diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index edf757e539a9..89df3a70963f 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -31,7 +31,7 @@ use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::demux::DemuxedStreamReceiver; use crate::datasource::file_format::write::BatchSerializer; use crate::datasource::physical_plan::{ - CsvExec, FileGroupDisplay, FileScanConfig, FileSink, FileSinkConfig, + CsvSource, FileGroupDisplay, FileScanConfig, FileSink, FileSinkConfig, }; use crate::error::Result; use crate::execution::context::SessionState; @@ -53,10 +53,11 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexRequirement; +use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::BoxStream; use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; @@ -410,9 +411,10 @@ impl FileFormat for CsvFormat { async fn create_physical_plan( &self, state: &SessionState, - conf: FileScanConfig, + mut conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { + conf.file_compression_type = self.options.compression.into(); // Consult configuration options for default values let has_header = self .options @@ -422,18 +424,17 @@ impl FileFormat for CsvFormat { .options .newlines_in_values .unwrap_or(state.config_options().catalog.newlines_in_values); + conf.new_lines_in_values = newlines_in_values; - let exec = CsvExec::builder(conf) - .with_has_header(has_header) - .with_delimeter(self.options.delimiter) - .with_quote(self.options.quote) - .with_terminator(self.options.terminator) - .with_escape(self.options.escape) - .with_comment(self.options.comment) - .with_newlines_in_values(newlines_in_values) - .with_file_compression_type(self.options.compression.into()) - .build(); - Ok(Arc::new(exec)) + let source = Arc::new( + CsvSource::new(has_header, self.options.delimiter, self.options.quote) + .with_escape(self.options.escape) + .with_terminator(self.options.terminator) + .with_comment(self.options.comment), + ); + conf = conf.with_source(source); + + Ok(conf.new_exec()) } async fn create_writer_physical_plan( @@ -472,6 +473,10 @@ impl FileFormat for CsvFormat { Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) } + + fn file_source(&self) -> Arc { + Arc::new(CsvSource::default()) + } } impl CsvFormat { @@ -1202,7 +1207,7 @@ mod tests { } /// Explain the `sql` query under `ctx` to make sure the underlying csv scan is parallelized - /// e.g. "CsvExec: file_groups={2 groups:" in plan means 2 CsvExec runs concurrently + /// e.g. "DataSourceExec: file_groups={2 groups:" in plan means 2 DataSourceExec runs concurrently async fn count_query_csv_partitions( ctx: &SessionContext, sql: &str, @@ -1211,7 +1216,7 @@ mod tests { let result = df.collect().await?; let plan = format!("{}", &pretty_format_batches(&result)?); - let re = Regex::new(r"CsvExec: file_groups=\{(\d+) group").unwrap(); + let re = Regex::new(r"DataSourceExec: file_groups=\{(\d+) group").unwrap(); if let Some(captures) = re.captures(&plan) { if let Some(match_) = captures.get(1) { @@ -1220,7 +1225,7 @@ mod tests { } } - internal_err!("query contains no CsvExec") + internal_err!("query contains no DataSourceExec") } #[rstest(n_partitions, case(1), case(2), case(3), case(4))] diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 5bffb7e582c1..d96efeec3895 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -32,8 +32,9 @@ use super::{ use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::demux::DemuxedStreamReceiver; use crate::datasource::file_format::write::BatchSerializer; -use crate::datasource::physical_plan::{FileGroupDisplay, FileSink}; -use crate::datasource::physical_plan::{FileSinkConfig, NdJsonExec}; +use crate::datasource::physical_plan::{ + FileGroupDisplay, FileSink, FileSinkConfig, JsonSource, +}; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::insert::{DataSink, DataSinkExec}; @@ -56,6 +57,7 @@ use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::ExecutionPlan; +use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; use datafusion_physical_expr_common::sort_expr::LexRequirement; @@ -246,12 +248,14 @@ impl FileFormat for JsonFormat { async fn create_physical_plan( &self, _state: &SessionState, - conf: FileScanConfig, + mut conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { - let exec = - NdJsonExec::new(conf, FileCompressionType::from(self.options.compression)); - Ok(Arc::new(exec)) + let source = Arc::new(JsonSource::new()); + conf.file_compression_type = FileCompressionType::from(self.options.compression); + conf = conf.with_source(source); + + Ok(conf.new_exec()) } async fn create_writer_physical_plan( @@ -271,6 +275,10 @@ impl FileFormat for JsonFormat { Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) } + + fn file_source(&self) -> Arc { + Arc::new(JsonSource::default()) + } } impl Default for JsonSerializer { diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 2e2e6dba1c0e..bb7e42d3f854 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -50,6 +50,7 @@ use datafusion_common::{internal_err, not_impl_err, GetExt}; use datafusion_expr::Expr; use datafusion_physical_expr::PhysicalExpr; +use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::{Buf, Bytes}; use datafusion_physical_expr_common::sort_expr::LexRequirement; @@ -154,6 +155,9 @@ pub trait FileFormat: Send + Sync + Debug { ) -> Result { Ok(FilePushdownSupport::NoSupport) } + + /// Return the related FileSource such as `CsvSource`, `JsonSource`, etc. + fn file_source(&self) -> Arc; } /// An enum to distinguish between different states when determining if certain filters can be @@ -595,11 +599,15 @@ pub(crate) mod test_util { let exec = format .create_physical_plan( state, - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) - .with_file_groups(file_groups) - .with_statistics(statistics) - .with_projection(projection) - .with_limit(limit), + FileScanConfig::new( + ObjectStoreUrl::local_filesystem(), + file_schema, + format.file_source(), + ) + .with_file_groups(file_groups) + .with_statistics(statistics) + .with_projection(projection) + .with_limit(limit), None, ) .await?; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 4c7169764a76..70121c96ae16 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -34,9 +34,8 @@ use crate::arrow::array::RecordBatch; use crate::arrow::datatypes::{Fields, Schema, SchemaRef}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::get_writer_schema; -use crate::datasource::physical_plan::parquet::{ - can_expr_be_pushed_down_with_schemas, ParquetExecBuilder, -}; +use crate::datasource::physical_plan::parquet::can_expr_be_pushed_down_with_schemas; +use crate::datasource::physical_plan::parquet::source::ParquetSource; use crate::datasource::physical_plan::{FileGroupDisplay, FileSink, FileSinkConfig}; use crate::datasource::statistics::{create_max_min_accs, get_col_stats}; use crate::error::Result; @@ -65,6 +64,7 @@ use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexRequirement; +use crate::datasource::data_source::FileSource; use async_trait::async_trait; use bytes::Bytes; use futures::future::BoxFuture; @@ -396,25 +396,34 @@ impl FileFormat for ParquetFormat { async fn create_physical_plan( &self, _state: &SessionState, - conf: FileScanConfig, + mut conf: FileScanConfig, filters: Option<&Arc>, ) -> Result> { - let mut builder = - ParquetExecBuilder::new_with_options(conf, self.options.clone()); + let mut predicate = None; + let mut metadata_size_hint = None; // 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. if self.enable_pruning() { - if let Some(predicate) = filters.cloned() { - builder = builder.with_predicate(predicate); + if let Some(pred) = filters.cloned() { + predicate = Some(pred); } } - if let Some(metadata_size_hint) = self.metadata_size_hint() { - builder = builder.with_metadata_size_hint(metadata_size_hint); + if let Some(metadata) = self.metadata_size_hint() { + metadata_size_hint = Some(metadata); } - Ok(builder.build_arc()) + let mut source = ParquetSource::new(self.options.clone()); + + if let Some(predicate) = predicate { + source = source.with_predicate(Arc::clone(&conf.file_schema), predicate); + } + if let Some(metadata_size_hint) = metadata_size_hint { + source = source.with_metadata_size_hint(metadata_size_hint) + } + conf = conf.with_source(Arc::new(source)); + Ok(conf.new_exec()) } async fn create_writer_physical_plan( @@ -453,6 +462,10 @@ impl FileFormat for ParquetFormat { FilePushdownSupport::NotSupportedForFilter }) } + + fn file_source(&self) -> Arc { + Arc::new(ParquetSource::default()) + } } /// [`MetadataFetch`] adapter for reading bytes from an [`ObjectStore`] diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 9b9bcd22c464..7d95a40186b2 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -933,14 +933,18 @@ impl TableProvider for ListingTable { .format .create_physical_plan( session_state, - FileScanConfig::new(object_store_url, Arc::clone(&self.file_schema)) - .with_file_groups(partitioned_file_lists) - .with_constraints(self.constraints.clone()) - .with_statistics(statistics) - .with_projection(projection.cloned()) - .with_limit(limit) - .with_output_ordering(output_ordering) - .with_table_partition_cols(table_partition_cols), + FileScanConfig::new( + object_store_url, + Arc::clone(&self.file_schema), + self.options.format.file_source(), + ) + .with_file_groups(partitioned_file_lists) + .with_constraints(self.constraints.clone()) + .with_statistics(statistics) + .with_projection(projection.cloned()) + .with_limit(limit) + .with_output_ordering(output_ordering) + .with_table_partition_cols(table_partition_cols), filters.as_ref(), ) .await diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 31239ed332ae..a996990105b3 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -27,7 +27,6 @@ use crate::error::Result; use crate::execution::context::SessionState; use crate::logical_expr::Expr; use crate::physical_plan::insert::{DataSink, DataSinkExec}; -use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::{ common, DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, @@ -42,6 +41,8 @@ use datafusion_common::{not_impl_err, plan_err, Constraints, DFSchema, SchemaExt use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_expr::SortExpr; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use async_trait::async_trait; use futures::StreamExt; @@ -162,7 +163,11 @@ impl MemTable { } } - let mut exec = MemoryExec::try_new(&data, Arc::clone(&schema), None)?; + let mut exec = DataSourceExec::new(Arc::new(MemorySourceConfig::try_new( + &data, + Arc::clone(&schema), + None, + )?)); if let Some(cons) = constraints { exec = exec.with_constraints(cons.clone()); } @@ -223,11 +228,11 @@ impl TableProvider for MemTable { partitions.push(inner_vec.clone()) } - let mut exec = - MemoryExec::try_new(&partitions, self.schema(), projection.cloned())?; + let mut source = + MemorySourceConfig::try_new(&partitions, self.schema(), projection.cloned())?; let show_sizes = state.config_options().explain.show_sizes; - exec = exec.with_show_sizes(show_sizes); + source = source.with_show_sizes(show_sizes); // add sort information if present let sort_order = self.sort_order.lock(); @@ -244,10 +249,10 @@ impl TableProvider for MemTable { ) }) .collect::>>()?; - exec = exec.try_with_sort_information(file_sort_order)?; + source = source.try_with_sort_information(file_sort_order)?; } - Ok(Arc::new(exec)) + Ok(Arc::new(DataSourceExec::new(Arc::new(source)))) } /// Returns an ExecutionPlan that inserts the execution results of a given [`ExecutionPlan`] into this [`MemTable`]. diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index aeb7d6569de9..55df55ae3543 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -21,6 +21,7 @@ pub mod avro_to_arrow; pub mod cte_worktable; +pub mod data_source; pub mod default_table_source; pub mod dynamic_file; pub mod empty; diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 82735334c7f8..5c5dce46e936 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,26 +20,27 @@ use std::any::Any; use std::sync::Arc; -use super::FileGroupPartitioner; +use crate::datasource::data_source::FileSource; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{ - FileMeta, FileOpenFuture, FileOpener, FileScanConfig, + FileMeta, FileOpenFuture, FileOpener, FileScanConfig, JsonSource, }; use crate::error::Result; -use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, -}; use arrow::buffer::Buffer; use arrow_ipc::reader::FileDecoder; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::PlanProperties; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::source::DataSourceExec; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; use futures::StreamExt; use itertools::Itertools; @@ -47,16 +48,13 @@ use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; /// Execution plan for scanning Arrow data source #[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] pub struct ArrowExec { + inner: DataSourceExec, base_config: FileScanConfig, - projected_statistics: Statistics, - projected_schema: SchemaRef, - projected_output_ordering: Vec, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - cache: PlanProperties, } +#[allow(unused, deprecated)] impl ArrowExec { /// Create a new Arrow reader execution plan provided base configurations pub fn new(base_config: FileScanConfig) -> Self { @@ -72,13 +70,11 @@ impl ArrowExec { projected_constraints, &base_config, ); + let arrow = ArrowSource::default(); + let base_config = base_config.with_source(Arc::new(arrow)); Self { + inner: DataSourceExec::new(Arc::new(base_config.clone())), base_config, - projected_schema, - projected_statistics, - projected_output_ordering, - metrics: ExecutionPlanMetricsSet::new(), - cache, } } /// Ref to the base configs @@ -86,6 +82,25 @@ impl ArrowExec { &self.base_config } + fn file_scan_config(&self) -> FileScanConfig { + let source = self.inner.source(); + source + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + fn json_source(&self) -> JsonSource { + let source = self.file_scan_config(); + source + .file_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) } @@ -111,25 +126,26 @@ impl ArrowExec { } fn with_file_groups(mut self, file_groups: Vec>) -> Self { - self.base_config.file_groups = file_groups; - // Changing file groups may invalidate output partitioning. Update it also - let output_partitioning = Self::output_partitioning_helper(&self.base_config); - self.cache = self.cache.with_partitioning(output_partitioning); + self.base_config.file_groups = file_groups.clone(); + let mut file_source = self.file_scan_config(); + file_source = file_source.with_file_groups(file_groups); + self.inner = self.inner.with_source(Arc::new(file_source)); self } } +#[allow(unused, deprecated)] impl DisplayAs for ArrowExec { fn fmt_as( &self, t: DisplayFormatType, f: &mut std::fmt::Formatter, ) -> std::fmt::Result { - write!(f, "ArrowExec: ")?; - self.base_config.fmt_as(t, f) + self.inner.fmt_as(t, f) } } +#[allow(unused, deprecated)] impl ExecutionPlan for ArrowExec { fn name(&self) -> &'static str { "ArrowExec" @@ -140,9 +156,8 @@ impl ExecutionPlan for ArrowExec { } fn properties(&self) -> &PlanProperties { - &self.cache + self.inner.properties() } - fn children(&self) -> Vec<&Arc> { Vec::new() } @@ -155,74 +170,95 @@ impl ExecutionPlan for ArrowExec { } /// Redistribute files across partitions according to their size - /// See comments on [`FileGroupPartitioner`] for more detail. + /// See comments on `FileGroupPartitioner` for more detail. fn repartitioned( &self, target_partitions: usize, config: &ConfigOptions, ) -> Result>> { - let repartition_file_min_size = config.optimizer.repartition_file_min_size; - let repartitioned_file_groups_option = FileGroupPartitioner::new() - .with_target_partitions(target_partitions) - .with_repartition_file_min_size(repartition_file_min_size) - .with_preserve_order_within_groups( - self.properties().output_ordering().is_some(), - ) - .repartition_file_groups(&self.base_config.file_groups); - - if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - let mut new_plan = self.clone(); - new_plan = new_plan.with_file_groups(repartitioned_file_groups); - return Ok(Some(Arc::new(new_plan))); - } - Ok(None) + self.inner.repartitioned(target_partitions, config) } - fn execute( &self, partition: usize, context: Arc, ) -> Result { - use super::file_stream::FileStream; - let object_store = context - .runtime_env() - .object_store(&self.base_config.object_store_url)?; - - let opener = ArrowOpener { - object_store, - projection: self.base_config.file_column_projection_indices(), - }; - let stream = - FileStream::new(&self.base_config, partition, opener, &self.metrics)?; - Ok(Box::pin(stream)) + self.inner.execute(partition, context) } - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + self.inner.metrics() } - fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) + self.inner.statistics() } - fn fetch(&self) -> Option { - self.base_config.limit + self.inner.fetch() } fn with_fetch(&self, limit: Option) -> Option> { - let new_config = self.base_config.clone().with_limit(limit); - - Some(Arc::new(Self { - base_config: new_config, - projected_statistics: self.projected_statistics.clone(), - projected_schema: Arc::clone(&self.projected_schema), - projected_output_ordering: self.projected_output_ordering.clone(), - metrics: self.metrics.clone(), - cache: self.cache.clone(), + self.inner.with_fetch(limit) + } +} + +/// Arrow configuration struct that is given to DataSourceExec +/// Does not hold anything special, since [`FileScanConfig`] is sufficient for arrow +#[derive(Clone, Default)] +pub struct ArrowSource { + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, +} + +impl FileSource for ArrowSource { + fn create_file_opener( + &self, + object_store: Result>, + base_config: &FileScanConfig, + _partition: usize, + ) -> Result> { + Ok(Arc::new(ArrowOpener { + object_store: object_store?, + projection: base_config.file_column_projection_indices(), })) } + + fn as_any(&self) -> &dyn Any { + self + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(Self { ..self.clone() }) + } + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut conf = self.clone(); + conf.projected_statistics = Some(statistics); + Arc::new(conf) + } + + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn statistics(&self) -> Result { + let statistics = &self.projected_statistics; + Ok(statistics + .clone() + .expect("projected_statistics must be set")) + } + + fn file_type(&self) -> &str { + "arrow" + } } +/// The struct arrow that implements `[FileOpener]` trait pub struct ArrowOpener { pub object_store: Arc, pub projection: Option>, diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 87d8964bed6a..ee3f7071c11f 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -18,34 +18,38 @@ //! Execution plan for reading line-delimited Avro files use std::any::Any; +use std::fmt::Formatter; use std::sync::Arc; -use super::FileScanConfig; +use super::{FileOpener, FileScanConfig}; +#[cfg(feature = "avro")] +use crate::datasource::avro_to_arrow::Reader as AvroReader; +use crate::datasource::data_source::FileSource; use crate::error::Result; -use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, Statistics, -}; use arrow::datatypes::SchemaRef; -use datafusion_common::Constraints; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_common::{Constraints, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::source::DataSourceExec; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; + +use object_store::ObjectStore; /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] pub struct AvroExec { + inner: DataSourceExec, base_config: FileScanConfig, - projected_statistics: Statistics, - projected_schema: SchemaRef, - projected_output_ordering: Vec, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - cache: PlanProperties, } +#[allow(unused, deprecated)] impl AvroExec { /// Create a new Avro reader execution plan provided base configurations pub fn new(base_config: FileScanConfig) -> Self { @@ -61,15 +65,13 @@ impl AvroExec { projected_constraints, &base_config, ); + let base_config = base_config.with_source(Arc::new(AvroSource::default())); Self { + inner: DataSourceExec::new(Arc::new(base_config.clone())), base_config, - projected_schema, - projected_statistics, - projected_output_ordering, - metrics: ExecutionPlanMetricsSet::new(), - cache, } } + /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config @@ -96,17 +98,14 @@ impl AvroExec { } } +#[allow(unused, deprecated)] impl DisplayAs for AvroExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - write!(f, "AvroExec: ")?; - self.base_config.fmt_as(t, f) + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + self.inner.fmt_as(t, f) } } +#[allow(unused, deprecated)] impl ExecutionPlan for AvroExec { fn name(&self) -> &'static str { "AvroExec" @@ -117,20 +116,17 @@ impl ExecutionPlan for AvroExec { } fn properties(&self) -> &PlanProperties { - &self.cache + self.inner.properties() } - fn children(&self) -> Vec<&Arc> { Vec::new() } - fn with_new_children( self: Arc, _: Vec>, ) -> Result> { Ok(self) } - #[cfg(not(feature = "avro"))] fn execute( &self, @@ -141,61 +137,129 @@ impl ExecutionPlan for AvroExec { "Cannot execute avro plan without avro feature enabled".to_string(), )) } - #[cfg(feature = "avro")] fn execute( &self, partition: usize, context: Arc, ) -> Result { - use super::file_stream::FileStream; - let object_store = context - .runtime_env() - .object_store(&self.base_config.object_store_url)?; - - let config = Arc::new(private::AvroConfig { - schema: Arc::clone(&self.base_config.file_schema), - batch_size: context.session_config().batch_size(), - projection: self.base_config.projected_file_column_names(), - object_store, - }); - let opener = private::AvroOpener { config }; - - let stream = - FileStream::new(&self.base_config, partition, opener, &self.metrics)?; - Ok(Box::pin(stream)) + self.inner.execute(partition, context) } fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) + self.inner.statistics() } fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + self.inner.metrics() } fn fetch(&self) -> Option { - self.base_config.limit + self.inner.fetch() } fn with_fetch(&self, limit: Option) -> Option> { - let new_config = self.base_config.clone().with_limit(limit); - - Some(Arc::new(Self { - base_config: new_config, - projected_statistics: self.projected_statistics.clone(), - projected_schema: Arc::clone(&self.projected_schema), - projected_output_ordering: self.projected_output_ordering.clone(), - metrics: self.metrics.clone(), - cache: self.cache.clone(), + self.inner.with_fetch(limit) + } +} + +/// AvroSource holds the extra configuration that is necessary for opening avro files +#[derive(Clone, Default)] +pub struct AvroSource { + schema: Option, + batch_size: Option, + projection: Option>, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, +} + +impl AvroSource { + /// Initialize an AvroSource with default values + pub fn new() -> Self { + Self::default() + } + + #[cfg(feature = "avro")] + fn open(&self, reader: R) -> Result> { + AvroReader::try_new( + reader, + Arc::clone(self.schema.as_ref().expect("Schema must set before open")), + self.batch_size.expect("Batch size must set before open"), + self.projection.clone(), + ) + } +} + +impl FileSource for AvroSource { + #[cfg(feature = "avro")] + fn create_file_opener( + &self, + object_store: Result>, + _base_config: &FileScanConfig, + _partition: usize, + ) -> Result> { + Ok(Arc::new(private::AvroOpener { + config: Arc::new(self.clone()), + object_store: object_store?, })) } + + #[cfg(not(feature = "avro"))] + fn create_file_opener( + &self, + _object_store: Result>, + _base_config: &FileScanConfig, + _partition: usize, + ) -> Result> { + panic!("Avro feature is not enabled in this build") + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) + } + + fn with_schema(&self, schema: SchemaRef) -> Arc { + let mut conf = self.clone(); + conf.schema = Some(schema); + Arc::new(conf) + } + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut conf = self.clone(); + conf.projected_statistics = Some(statistics); + Arc::new(conf) + } + + fn with_projection(&self, config: &FileScanConfig) -> Arc { + let mut conf = self.clone(); + conf.projection = config.projected_file_column_names(); + Arc::new(conf) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn statistics(&self) -> Result { + let statistics = &self.projected_statistics; + Ok(statistics + .clone() + .expect("projected_statistics must be set")) + } + + fn file_type(&self) -> &str { + "avro" + } } #[cfg(feature = "avro")] mod private { use super::*; - use crate::datasource::avro_to_arrow::Reader as AvroReader; use crate::datasource::physical_plan::file_stream::{FileOpenFuture, FileOpener}; use crate::datasource::physical_plan::FileMeta; @@ -203,14 +267,14 @@ mod private { use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; - pub struct AvroConfig { + pub struct DeprecatedAvroConfig { pub schema: SchemaRef, pub batch_size: usize, pub projection: Option>, pub object_store: Arc, } - impl AvroConfig { + impl DeprecatedAvroConfig { fn open(&self, reader: R) -> Result> { AvroReader::try_new( reader, @@ -221,15 +285,40 @@ mod private { } } + pub struct DeprecatedAvroOpener { + pub config: Arc, + } + impl FileOpener for DeprecatedAvroOpener { + fn open(&self, file_meta: FileMeta) -> Result { + let config = Arc::clone(&self.config); + Ok(Box::pin(async move { + let r = config.object_store.get(file_meta.location()).await?; + match r.payload { + GetResultPayload::File(file, _) => { + let reader = config.open(file)?; + Ok(futures::stream::iter(reader).boxed()) + } + GetResultPayload::Stream(_) => { + let bytes = r.bytes().await?; + let reader = config.open(bytes.reader())?; + Ok(futures::stream::iter(reader).boxed()) + } + } + })) + } + } + pub struct AvroOpener { - pub config: Arc, + pub config: Arc, + pub object_store: Arc, } impl FileOpener for AvroOpener { fn open(&self, file_meta: FileMeta) -> Result { let config = Arc::clone(&self.config); + let object_store = Arc::clone(&self.object_store); Ok(Box::pin(async move { - let r = config.object_store.get(file_meta.location()).await?; + let r = object_store.get(file_meta.location()).await?; match r.payload { GetResultPayload::File(file, _) => { let reader = config.open(file)?; @@ -258,6 +347,8 @@ mod tests { use crate::scalar::ScalarValue; use crate::test::object_store::local_unpartitioned_file; + use datafusion_physical_plan::ExecutionPlan; + use futures::StreamExt; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; @@ -297,19 +388,21 @@ mod tests { .infer_schema(&state, &store, std::slice::from_ref(&meta)) .await?; - let avro_exec = AvroExec::new( - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) + let source = Arc::new(AvroSource::new()); + let conf = + FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema, source) .with_file(meta.into()) - .with_projection(Some(vec![0, 1, 2])), - ); + .with_projection(Some(vec![0, 1, 2])); + + let source_exec = conf.new_exec(); assert_eq!( - avro_exec + source_exec .properties() .output_partitioning() .partition_count(), 1 ); - let mut results = avro_exec + let mut results = source_exec .execute(0, state.task_ctx()) .expect("plan execution failed"); @@ -369,20 +462,21 @@ mod tests { // Include the missing column in the projection let projection = Some(vec![0, 1, 2, actual_schema.fields().len()]); - let avro_exec = AvroExec::new( - FileScanConfig::new(object_store_url, file_schema) - .with_file(meta.into()) - .with_projection(projection), - ); + let source = Arc::new(AvroSource::new()); + let conf = FileScanConfig::new(object_store_url, file_schema, source) + .with_file(meta.into()) + .with_projection(projection); + + let source_exec = conf.new_exec(); assert_eq!( - avro_exec + source_exec .properties() .output_partitioning() .partition_count(), 1 ); - let mut results = avro_exec + let mut results = source_exec .execute(0, state.task_ctx()) .expect("plan execution failed"); @@ -439,27 +533,25 @@ mod tests { partitioned_file.partition_values = vec![ScalarValue::from("2021-10-26")]; let projection = Some(vec![0, 1, file_schema.fields().len(), 2]); - let avro_exec = AvroExec::new( - FileScanConfig::new(object_store_url, file_schema) - // select specific columns of the files as well as the partitioning - // column which is supposed to be the last column in the table schema. - .with_projection(projection) - .with_file(partitioned_file) - .with_table_partition_cols(vec![Field::new( - "date", - DataType::Utf8, - false, - )]), - ); + let source = Arc::new(AvroSource::new()); + let conf = FileScanConfig::new(object_store_url, file_schema, source) + // select specific columns of the files as well as the partitioning + // column which is supposed to be the last column in the table schema. + .with_projection(projection) + .with_file(partitioned_file) + .with_table_partition_cols(vec![Field::new("date", DataType::Utf8, false)]); + + let source_exec = conf.new_exec(); + assert_eq!( - avro_exec + source_exec .properties() .output_partitioning() .partition_count(), 1 ); - let mut results = avro_exec + let mut results = source_exec .execute(0, state.task_ctx()) .expect("plan execution failed"); diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 55971f6f627c..269af4df903a 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -18,85 +18,55 @@ //! Execution plan for reading CSV files use std::any::Any; +use std::fmt; use std::io::{Read, Seek, SeekFrom}; use std::sync::Arc; use std::task::Poll; -use super::{calculate_range, FileGroupPartitioner, FileScanConfig, RangeCalculation}; +use super::{calculate_range, FileScanConfig, RangeCalculation}; +use crate::datasource::data_source::FileSource; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; use crate::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; -use crate::datasource::physical_plan::file_stream::{ - FileOpenFuture, FileOpener, FileStream, -}; +use crate::datasource::physical_plan::file_stream::{FileOpenFuture, FileOpener}; use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; -use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, Partitioning, - PlanProperties, SendableRecordBatchStream, Statistics, -}; +use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::Constraints; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; - +use datafusion_common::{Constraints, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::projection::{ - all_alias_free_columns, new_projections_for_columns, ProjectionExec, -}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::source::DataSourceExec; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; + use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; use tokio::task::JoinSet; -/// Execution plan for scanning a CSV file. +/// Old Csv source, deprecated with DataSourceExec implementation and CsvSource /// -/// # Example: create a `CsvExec` -/// ``` -/// # use std::sync::Arc; -/// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::{ -/// # physical_plan::{CsvExec, FileScanConfig}, -/// # listing::PartitionedFile, -/// # }; -/// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # let object_store_url = ObjectStoreUrl::local_filesystem(); -/// # let file_schema = Arc::new(Schema::empty()); -/// // Create a CsvExec for reading the first 100MB of `file1.csv` -/// let file_scan_config = FileScanConfig::new(object_store_url, file_schema) -/// .with_file(PartitionedFile::new("file1.csv", 100*1024*1024)); -/// let exec = CsvExec::builder(file_scan_config) -/// .with_has_header(true) // The file has a header row -/// .with_newlines_in_values(true) // The file contains newlines in values -/// .build(); -/// ``` +/// See examples on `CsvSource` #[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] pub struct CsvExec { base_config: FileScanConfig, - projected_statistics: Statistics, - has_header: bool, - delimiter: u8, - quote: u8, - terminator: Option, - escape: Option, - comment: Option, - newlines_in_values: bool, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - /// Compression type of the file associated with CsvExec - pub file_compression_type: FileCompressionType, - cache: PlanProperties, + inner: DataSourceExec, } /// Builder for [`CsvExec`]. /// /// See example on [`CsvExec`]. #[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use FileScanConfig instead")] pub struct CsvExecBuilder { file_scan_config: FileScanConfig, file_compression_type: FileCompressionType, @@ -110,6 +80,7 @@ pub struct CsvExecBuilder { newlines_in_values: bool, } +#[allow(unused, deprecated)] impl CsvExecBuilder { /// Create a new builder to read the provided file scan configuration. pub fn new(file_scan_config: FileScanConfig) -> Self { @@ -225,27 +196,25 @@ impl CsvExecBuilder { projected_constraints, &base_config, ); + let csv = CsvSource::new(has_header, delimiter, quote) + .with_comment(comment) + .with_escape(escape) + .with_terminator(terminator); + let base_config = base_config + .with_newlines_in_values(newlines_in_values) + .with_file_compression_type(file_compression_type) + .with_source(Arc::new(csv)); CsvExec { + inner: DataSourceExec::new(Arc::new(base_config.clone())), base_config, - projected_statistics, - has_header, - delimiter, - quote, - terminator, - escape, - newlines_in_values, - metrics: ExecutionPlanMetricsSet::new(), - file_compression_type, - cache, - comment, } } } +#[allow(unused, deprecated)] impl CsvExec { /// Create a new CSV reader execution plan provided base and specific configurations - #[deprecated(since = "41.0.0", note = "use `CsvExec::builder` or `CsvExecBuilder`")] #[allow(clippy::too_many_arguments)] pub fn new( base_config: FileScanConfig, @@ -281,33 +250,29 @@ impl CsvExec { pub fn base_config(&self) -> &FileScanConfig { &self.base_config } - /// true if the first line of each file is a header - pub fn has_header(&self) -> bool { - self.has_header - } - /// A column delimiter - pub fn delimiter(&self) -> u8 { - self.delimiter - } - - /// The quote character - pub fn quote(&self) -> u8 { - self.quote - } - /// The line terminator - pub fn terminator(&self) -> Option { - self.terminator + fn file_scan_config(&self) -> FileScanConfig { + let source = self.inner.source(); + source + .as_any() + .downcast_ref::() + .unwrap() + .clone() } - /// Lines beginning with this byte are ignored. - pub fn comment(&self) -> Option { - self.comment + fn csv_source(&self) -> CsvSource { + let source = self.file_scan_config(); + source + .file_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() } - /// The escape character - pub fn escape(&self) -> Option { - self.escape + /// true if the first line of each file is a header + pub fn has_header(&self) -> bool { + self.csv_source().has_header() } /// Specifies whether newlines in (quoted) values are supported. @@ -318,7 +283,8 @@ impl CsvExec { /// /// The default behaviour depends on the `datafusion.catalog.newlines_in_values` setting. pub fn newlines_in_values(&self) -> bool { - self.newlines_in_values + let source = self.file_scan_config(); + source.newlines_in_values() } fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { @@ -345,26 +311,22 @@ impl CsvExec { } fn with_file_groups(mut self, file_groups: Vec>) -> Self { - self.base_config.file_groups = file_groups; - // Changing file groups may invalidate output partitioning. Update it also - let output_partitioning = Self::output_partitioning_helper(&self.base_config); - self.cache = self.cache.with_partitioning(output_partitioning); + self.base_config.file_groups = file_groups.clone(); + let mut file_source = self.file_scan_config(); + file_source = file_source.with_file_groups(file_groups); + self.inner = self.inner.with_source(Arc::new(file_source)); self } } +#[allow(unused, deprecated)] impl DisplayAs for CsvExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - write!(f, "CsvExec: ")?; - self.base_config.fmt_as(t, f)?; - write!(f, ", has_header={}", self.has_header) + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + self.inner.fmt_as(t, f) } } +#[allow(unused, deprecated)] impl ExecutionPlan for CsvExec { fn name(&self) -> &'static str { "CsvExec" @@ -376,7 +338,7 @@ impl ExecutionPlan for CsvExec { } fn properties(&self) -> &PlanProperties { - &self.cache + self.inner.properties() } fn children(&self) -> Vec<&Arc> { @@ -392,7 +354,7 @@ impl ExecutionPlan for CsvExec { } /// Redistribute files across partitions according to their size - /// See comments on [`FileGroupPartitioner`] for more detail. + /// See comments on `FileGroupPartitioner` for more detail. /// /// Return `None` if can't get repartitioned (empty, compressed file, or `newlines_in_values` set). fn repartitioned( @@ -400,26 +362,7 @@ impl ExecutionPlan for CsvExec { target_partitions: usize, config: &ConfigOptions, ) -> Result>> { - let repartition_file_min_size = config.optimizer.repartition_file_min_size; - // Parallel execution on compressed CSV files or files that must support newlines in values is not supported yet. - if self.file_compression_type.is_compressed() || self.newlines_in_values { - return Ok(None); - } - - let repartitioned_file_groups_option = FileGroupPartitioner::new() - .with_target_partitions(target_partitions) - .with_preserve_order_within_groups( - self.properties().output_ordering().is_some(), - ) - .with_repartition_file_min_size(repartition_file_min_size) - .repartition_file_groups(&self.base_config.file_groups); - - if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - let mut new_plan = self.clone(); - new_plan = new_plan.with_file_groups(repartitioned_file_groups); - return Ok(Some(Arc::new(new_plan))); - } - Ok(None) + self.inner.repartitioned(target_partitions, config) } fn execute( @@ -427,148 +370,158 @@ impl ExecutionPlan for CsvExec { partition: usize, context: Arc, ) -> Result { - let object_store = context - .runtime_env() - .object_store(&self.base_config.object_store_url)?; - - let config = Arc::new(CsvConfig { - batch_size: context.session_config().batch_size(), - file_schema: Arc::clone(&self.base_config.file_schema), - file_projection: self.base_config.file_column_projection_indices(), - has_header: self.has_header, - delimiter: self.delimiter, - quote: self.quote, - escape: self.escape, - terminator: self.terminator, - object_store, - comment: self.comment, - }); - let opener = CsvOpener { - config, - file_compression_type: self.file_compression_type.to_owned(), - }; - let stream = - FileStream::new(&self.base_config, partition, opener, &self.metrics)?; - Ok(Box::pin(stream) as SendableRecordBatchStream) + self.inner.execute(partition, context) } fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) + self.inner.statistics() } fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + self.inner.metrics() } fn fetch(&self) -> Option { - self.base_config.limit + self.inner.fetch() } fn with_fetch(&self, limit: Option) -> Option> { - let new_config = self.base_config.clone().with_limit(limit); - - Some(Arc::new(Self { - base_config: new_config, - projected_statistics: self.projected_statistics.clone(), - has_header: self.has_header, - delimiter: self.delimiter, - quote: self.quote, - escape: self.escape, - terminator: self.terminator, - comment: self.comment, - newlines_in_values: self.newlines_in_values, - metrics: self.metrics.clone(), - file_compression_type: self.file_compression_type, - cache: self.cache.clone(), - })) + self.inner.with_fetch(limit) } fn try_swapping_with_projection( &self, projection: &ProjectionExec, ) -> Result>> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into CsvExec, but it would be an overlap of their responsibility. - Ok(all_alias_free_columns(projection.expr()).then(|| { - let mut file_scan = self.base_config().clone(); - let new_projections = new_projections_for_columns( - projection, - &file_scan - .projection - .unwrap_or((0..self.schema().fields().len()).collect()), - ); - file_scan.projection = Some(new_projections); - - Arc::new( - CsvExec::builder(file_scan) - .with_has_header(self.has_header()) - .with_delimeter(self.delimiter()) - .with_quote(self.quote()) - .with_escape(self.escape()) - .with_comment(self.comment()) - .with_newlines_in_values(self.newlines_in_values()) - .with_file_compression_type(self.file_compression_type) - .build(), - ) as _ - })) + self.inner.try_swapping_with_projection(projection) } } /// A Config for [`CsvOpener`] -#[derive(Debug, Clone)] -pub struct CsvConfig { - batch_size: usize, - file_schema: SchemaRef, +/// +/// # Example: create a `DataSourceExec` for CSV +/// ``` +/// # use std::sync::Arc; +/// # use arrow::datatypes::Schema; +/// # use datafusion::datasource::{ +/// # physical_plan::FileScanConfig, +/// # listing::PartitionedFile, +/// # }; +/// # use datafusion::datasource::physical_plan::CsvSource; +/// # use datafusion_execution::object_store::ObjectStoreUrl; +/// # use datafusion_physical_plan::source::DataSourceExec; +/// +/// # let object_store_url = ObjectStoreUrl::local_filesystem(); +/// # let file_schema = Arc::new(Schema::empty()); +/// +/// let source = Arc::new(CsvSource::new( +/// true, +/// b',', +/// b'"', +/// ) +/// .with_terminator(Some(b'#') +/// )); +/// // Create a DataSourceExec for reading the first 100MB of `file1.csv` +/// let file_scan_config = FileScanConfig::new(object_store_url, file_schema, source) +/// .with_file(PartitionedFile::new("file1.csv", 100*1024*1024)) +/// .with_newlines_in_values(true); // The file contains newlines in values; +/// let exec = file_scan_config.new_exec(); +/// ``` +#[derive(Debug, Clone, Default)] +pub struct CsvSource { + batch_size: Option, + file_schema: Option, file_projection: Option>, - has_header: bool, + pub(crate) has_header: bool, delimiter: u8, quote: u8, terminator: Option, escape: Option, - object_store: Arc, comment: Option, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, } -impl CsvConfig { - #[allow(clippy::too_many_arguments)] - /// Returns a [`CsvConfig`] - pub fn new( - batch_size: usize, - file_schema: SchemaRef, - file_projection: Option>, - has_header: bool, - delimiter: u8, - quote: u8, - terminator: Option, - object_store: Arc, - comment: Option, - ) -> Self { +impl CsvSource { + /// Returns a [`CsvSource`] + pub fn new(has_header: bool, delimiter: u8, quote: u8) -> Self { Self { - batch_size, - file_schema, - file_projection, has_header, delimiter, quote, - terminator, - escape: None, - object_store, - comment, + ..Self::default() } } + + /// true if the first line of each file is a header + pub fn has_header(&self) -> bool { + self.has_header + } + /// A column delimiter + pub fn delimiter(&self) -> u8 { + self.delimiter + } + + /// The quote character + pub fn quote(&self) -> u8 { + self.quote + } + + /// The line terminator + pub fn terminator(&self) -> Option { + self.terminator + } + + /// Lines beginning with this byte are ignored. + pub fn comment(&self) -> Option { + self.comment + } + + /// The escape character + pub fn escape(&self) -> Option { + self.escape + } + + /// Initialize a CsvSource with escape + pub fn with_escape(&self, escape: Option) -> Self { + let mut conf = self.clone(); + conf.escape = escape; + conf + } + + /// Initialize a CsvSource with terminator + pub fn with_terminator(&self, terminator: Option) -> Self { + let mut conf = self.clone(); + conf.terminator = terminator; + conf + } + + /// Initialize a CsvSource with comment + pub fn with_comment(&self, comment: Option) -> Self { + let mut conf = self.clone(); + conf.comment = comment; + conf + } } -impl CsvConfig { +impl CsvSource { fn open(&self, reader: R) -> Result> { Ok(self.builder().build(reader)?) } fn builder(&self) -> csv::ReaderBuilder { - let mut builder = csv::ReaderBuilder::new(Arc::clone(&self.file_schema)) - .with_delimiter(self.delimiter) - .with_batch_size(self.batch_size) - .with_header(self.has_header) - .with_quote(self.quote); + let mut builder = csv::ReaderBuilder::new(Arc::clone( + self.file_schema + .as_ref() + .expect("Schema must be set before initializing builder"), + )) + .with_delimiter(self.delimiter) + .with_batch_size( + self.batch_size + .expect("Batch size must be set before initializing builder"), + ) + .with_header(self.has_header) + .with_quote(self.quote); if let Some(terminator) = self.terminator { builder = builder.with_terminator(terminator); } @@ -588,23 +541,85 @@ impl CsvConfig { /// A [`FileOpener`] that opens a CSV file and yields a [`FileOpenFuture`] pub struct CsvOpener { - config: Arc, + config: Arc, file_compression_type: FileCompressionType, + object_store: Arc, } impl CsvOpener { /// Returns a [`CsvOpener`] pub fn new( - config: Arc, + config: Arc, file_compression_type: FileCompressionType, + object_store: Arc, ) -> Self { Self { config, file_compression_type, + object_store, } } } +impl FileSource for CsvSource { + fn create_file_opener( + &self, + object_store: Result>, + base_config: &FileScanConfig, + _partition: usize, + ) -> Result> { + Ok(Arc::new(CsvOpener { + config: Arc::new(self.clone()), + file_compression_type: base_config.file_compression_type, + object_store: object_store?, + })) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) + } + + fn with_schema(&self, schema: SchemaRef) -> Arc { + let mut conf = self.clone(); + conf.file_schema = Some(schema); + Arc::new(conf) + } + + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut conf = self.clone(); + conf.projected_statistics = Some(statistics); + Arc::new(conf) + } + + fn with_projection(&self, config: &FileScanConfig) -> Arc { + let mut conf = self.clone(); + conf.file_projection = config.file_column_projection_indices(); + Arc::new(conf) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + fn statistics(&self) -> Result { + let statistics = &self.projected_statistics; + Ok(statistics + .clone() + .expect("projected_statistics must be set")) + } + fn file_type(&self) -> &str { + "csv" + } + fn fmt_extra(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, ", has_header={}", self.has_header) + } +} + impl FileOpener for CsvOpener { /// Open a partitioned CSV file. /// @@ -640,7 +655,7 @@ impl FileOpener for CsvOpener { } } - let config = CsvConfig { + let config = CsvSource { has_header: csv_has_header, ..(*self.config).clone() }; @@ -654,7 +669,7 @@ impl FileOpener for CsvOpener { ); } - let store = Arc::clone(&self.config.object_store); + let store = Arc::clone(&self.object_store); let terminator = self.config.terminator; Ok(Box::pin(async move { @@ -778,9 +793,8 @@ mod tests { use arrow::datatypes::*; use bytes::Bytes; use datafusion_common::test_util::arrow_test_data; + use datafusion_physical_plan::metrics::MetricsSet; - use datafusion_common::config::CsvOptions; - use datafusion_execution::object_store::ObjectStoreUrl; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; @@ -818,20 +832,15 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); + let source = Arc::new(CsvSource::new(true, b',', b'"')); + let mut config = partitioned_csv_config(file_schema, file_groups, source) + .with_file_compression_type(file_compression_type) + .with_newlines_in_values(false); config.projection = Some(vec![0, 2, 4]); - let csv = CsvExec::builder(config) - .with_has_header(true) - .with_delimeter(b',') - .with_quote(b'"') - .with_terminator(None) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type) - .build(); - assert_eq!(13, csv.base_config.file_schema.fields().len()); + let csv = config.new_exec(); + + assert_eq!(13, config.file_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); let mut stream = csv.execute(0, task_ctx)?; @@ -888,20 +897,13 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); - config.projection = Some(vec![4, 0, 2]); - - let csv = CsvExec::builder(config) - .with_has_header(true) - .with_delimeter(b',') - .with_quote(b'"') - .with_terminator(None) - .with_escape(None) - .with_comment(None) + let source = Arc::new(CsvSource::new(true, b',', b'"')); + let mut config = partitioned_csv_config(file_schema, file_groups, source) .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .build(); - assert_eq!(13, csv.base_config.file_schema.fields().len()); + .with_file_compression_type(file_compression_type.to_owned()); + config.projection = Some(vec![4, 0, 2]); + let csv = config.new_exec(); + assert_eq!(13, config.file_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); let mut stream = csv.execute(0, task_ctx)?; @@ -958,20 +960,13 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); - config.limit = Some(5); - - let csv = CsvExec::builder(config) - .with_has_header(true) - .with_delimeter(b',') - .with_quote(b'"') - .with_terminator(None) - .with_escape(None) - .with_comment(None) + let source = Arc::new(CsvSource::new(true, b',', b'"')); + let mut config = partitioned_csv_config(file_schema, file_groups, source) .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .build(); - assert_eq!(13, csv.base_config.file_schema.fields().len()); + .with_file_compression_type(file_compression_type.to_owned()); + config.limit = Some(5); + let csv = config.new_exec(); + assert_eq!(13, config.file_schema.fields().len()); assert_eq!(13, csv.schema().fields().len()); let mut it = csv.execute(0, task_ctx)?; @@ -1025,20 +1020,13 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); - config.limit = Some(5); - - let csv = CsvExec::builder(config) - .with_has_header(true) - .with_delimeter(b',') - .with_quote(b'"') - .with_terminator(None) - .with_escape(None) - .with_comment(None) + let source = Arc::new(CsvSource::new(true, b',', b'"')); + let mut config = partitioned_csv_config(file_schema, file_groups, source) .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .build(); - assert_eq!(14, csv.base_config.file_schema.fields().len()); + .with_file_compression_type(file_compression_type.to_owned()); + config.limit = Some(5); + let csv = config.new_exec(); + assert_eq!(14, config.file_schema.fields().len()); assert_eq!(14, csv.schema().fields().len()); // errors due to https://github.com/apache/datafusion/issues/4918 @@ -1082,7 +1070,10 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); + let source = Arc::new(CsvSource::new(true, b',', b'"')); + let mut config = partitioned_csv_config(file_schema, file_groups, source) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()); // Add partition columns config.table_partition_cols = vec![Field::new("date", DataType::Utf8, false)]; @@ -1094,17 +1085,9 @@ mod tests { // we don't have `/date=xx/` in the path but that is ok because // partitions are resolved during scan anyway - let csv = CsvExec::builder(config) - .with_has_header(true) - .with_delimeter(b',') - .with_quote(b'"') - .with_terminator(None) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .build(); - assert_eq!(13, csv.base_config.file_schema.fields().len()); + + let csv = config.new_exec(); + assert_eq!(13, config.file_schema.fields().len()); assert_eq!(2, csv.schema().fields().len()); let mut it = csv.execute(0, task_ctx)?; @@ -1189,17 +1172,11 @@ mod tests { ) .unwrap(); - let config = partitioned_csv_config(file_schema, file_groups); - let csv = CsvExec::builder(config) - .with_has_header(true) - .with_delimeter(b',') - .with_quote(b'"') - .with_terminator(None) - .with_escape(None) - .with_comment(None) + let source = Arc::new(CsvSource::new(true, b',', b'"')); + let config = partitioned_csv_config(file_schema, file_groups, source) .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .build(); + .with_file_compression_type(file_compression_type.to_owned()); + let csv = config.new_exec(); let it = csv.execute(0, task_ctx).unwrap(); let batches: Vec<_> = it.try_collect().await.unwrap(); @@ -1508,36 +1485,4 @@ mod tests { Arc::new(schema) } - - /// Ensure that the default options are set correctly - #[test] - fn test_default_options() { - let file_scan_config = - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), aggr_test_schema()) - .with_file(PartitionedFile::new("foo", 34)); - - let CsvExecBuilder { - file_scan_config: _, - file_compression_type: _, - has_header, - delimiter, - quote, - terminator, - escape, - comment, - newlines_in_values, - } = CsvExecBuilder::new(file_scan_config); - - let default_options = CsvOptions::default(); - assert_eq!(has_header, default_options.has_header.unwrap_or(false)); - assert_eq!(delimiter, default_options.delimiter); - assert_eq!(quote, default_options.quote); - assert_eq!(terminator, default_options.terminator); - assert_eq!(escape, default_options.escape); - assert_eq!(comment, default_options.comment); - assert_eq!( - newlines_in_values, - default_options.newlines_in_values.unwrap_or(false) - ); - } } diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 5a38886bb16f..652632c31554 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -18,14 +18,19 @@ //! [`FileScanConfig`] to configure scanning of possibly partitioned //! file sources. -use std::{ - borrow::Cow, collections::HashMap, fmt::Debug, marker::PhantomData, mem::size_of, - sync::Arc, vec, +use super::{ + get_projected_output_ordering, statistics::MinMaxStatistics, AvroSource, + FileGroupPartitioner, FileGroupsDisplay, FileStream, }; - -use super::{get_projected_output_ordering, statistics::MinMaxStatistics}; +use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl}; use crate::{error::Result, scalar::ScalarValue}; +use std::any::Any; +use std::fmt::Formatter; +use std::{ + borrow::Cow, collections::HashMap, fmt, fmt::Debug, marker::PhantomData, + mem::size_of, sync::Arc, vec, +}; use arrow::array::{ArrayData, BufferBuilder}; use arrow::buffer::Buffer; @@ -36,8 +41,17 @@ use datafusion_common::stats::Precision; use datafusion_common::{ exec_err, ColumnStatistics, Constraints, DataFusionError, Statistics, }; -use datafusion_physical_expr::LexOrdering; - +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, Partitioning}; + +use crate::datasource::data_source::FileSource; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_plan::display::{display_orderings, ProjectSchemaDisplay}; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_physical_plan::projection::{ + all_alias_free_columns, new_projections_for_columns, ProjectionExec, +}; +use datafusion_physical_plan::source::{DataSource, DataSourceExec}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use log::warn; /// Convert type to a type suitable for use as a [`ListingTable`] @@ -73,10 +87,11 @@ pub fn wrap_partition_value_in_dict(val: ScalarValue) -> ScalarValue { /// use datafusion::datasource::listing::PartitionedFile; /// # use datafusion::datasource::physical_plan::FileScanConfig; /// # use datafusion_execution::object_store::ObjectStoreUrl; +/// # use datafusion::datasource::physical_plan::ArrowSource; /// # let file_schema = Arc::new(Schema::empty()); /// // create FileScan config for reading data from file:// /// let object_store_url = ObjectStoreUrl::local_filesystem(); -/// let config = FileScanConfig::new(object_store_url, file_schema) +/// let config = FileScanConfig::new(object_store_url, file_schema, Arc::new(ArrowSource::default())) /// .with_limit(Some(1000)) // read only the first 1000 records /// .with_projection(Some(vec![2, 3])) // project columns 2 and 3 /// // Read /tmp/file1.parquet with known size of 1234 bytes in a single group @@ -131,22 +146,157 @@ pub struct FileScanConfig { pub table_partition_cols: Vec, /// All equivalent lexicographical orderings that describe the schema. pub output_ordering: Vec, + /// File compression type + pub file_compression_type: FileCompressionType, + /// Are new lines in values supported for CSVOptions + pub new_lines_in_values: bool, + /// File source such as `ParquetSource`, `CsvSource`, `JsonSource`, etc. + pub source: Arc, +} + +impl DataSource for FileScanConfig { + fn open( + &self, + partition: usize, + context: Arc, + ) -> Result { + let object_store = context.runtime_env().object_store(&self.object_store_url); + + let source = self + .source + .with_batch_size(context.session_config().batch_size()) + .with_schema(Arc::clone(&self.file_schema)) + .with_projection(self); + + let opener = source.create_file_opener(object_store, self, partition)?; + + let stream = FileStream::new(self, partition, opener, source.metrics())?; + Ok(Box::pin(stream)) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + let (schema, _, _, orderings) = self.project(); + + write!(f, "file_groups=")?; + FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + + if !schema.fields().is_empty() { + write!(f, ", projection={}", ProjectSchemaDisplay(&schema))?; + } + + if let Some(limit) = self.limit { + write!(f, ", limit={limit}")?; + } + + display_orderings(f, &orderings)?; + + if !self.constraints.is_empty() { + write!(f, ", {}", self.constraints)?; + } + + self.fmt_file_source(t, f) + } + + /// Redistribute files across partitions according to their size + /// See comments on [`FileGroupPartitioner`] for more detail. + fn repartitioned( + &self, + target_partitions: usize, + repartition_file_min_size: usize, + output_ordering: Option, + ) -> Result>> { + if !self.supports_repartition() { + return Ok(None); + } + + let repartitioned_file_groups_option = FileGroupPartitioner::new() + .with_target_partitions(target_partitions) + .with_repartition_file_min_size(repartition_file_min_size) + .with_preserve_order_within_groups(output_ordering.is_some()) + .repartition_file_groups(&self.file_groups); + + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + let mut source = self.clone(); + source.file_groups = repartitioned_file_groups; + return Ok(Some(Arc::new(source))); + } + Ok(None) + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.file_groups.len()) + } + + fn eq_properties(&self) -> EquivalenceProperties { + let (schema, constraints, _, orderings) = self.project(); + EquivalenceProperties::new_with_orderings(schema, orderings.as_slice()) + .with_constraints(constraints) + } + + fn statistics(&self) -> Result { + self.source.statistics() + } + + fn with_fetch(&self, limit: Option) -> Option> { + let source = self.clone(); + Some(Arc::new(source.with_limit(limit))) + } + + fn fetch(&self) -> Option { + self.limit + } + + fn metrics(&self) -> ExecutionPlanMetricsSet { + self.source.metrics().clone() + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into CsvExec, but it would be an overlap of their responsibility. + Ok(all_alias_free_columns(projection.expr()).then(|| { + let mut file_scan = self.clone(); + let source = Arc::clone(&file_scan.source); + let new_projections = new_projections_for_columns( + projection, + &file_scan + .projection + .unwrap_or((0..self.file_schema.fields().len()).collect()), + ); + file_scan.projection = Some(new_projections); + // Assign projected statistics to source + file_scan = file_scan.with_source(source); + + file_scan.new_exec() as _ + })) + } } impl FileScanConfig { - /// Create a new `FileScanConfig` with default settings for scanning files. + /// Create a new [`FileScanConfig`] with default settings for scanning files. /// /// See example on [`FileScanConfig`] /// - /// No file groups are added by default. See [`Self::with_file`], [`Self::with_file_group]` and + /// No file groups are added by default. See [`Self::with_file`], [`Self::with_file_group`] and /// [`Self::with_file_groups`]. /// /// # Parameters: /// * `object_store_url`: See [`Self::object_store_url`] /// * `file_schema`: See [`Self::file_schema`] - pub fn new(object_store_url: ObjectStoreUrl, file_schema: SchemaRef) -> Self { + pub fn new( + object_store_url: ObjectStoreUrl, + file_schema: SchemaRef, + file_source: Arc, + ) -> Self { let statistics = Statistics::new_unknown(&file_schema); - Self { + + let mut config = Self { object_store_url, file_schema, file_groups: vec![], @@ -156,7 +306,25 @@ impl FileScanConfig { limit: None, table_partition_cols: vec![], output_ordering: vec![], - } + file_compression_type: FileCompressionType::UNCOMPRESSED, + new_lines_in_values: false, + source: Arc::clone(&file_source), + }; + + config = config.with_source(Arc::clone(&file_source)); + config + } + + /// Set the file source + pub fn with_source(mut self, source: Arc) -> Self { + let ( + _projected_schema, + _constraints, + projected_statistics, + _projected_output_ordering, + ) = self.project(); + self.source = source.with_statistics(projected_statistics); + self } /// Set the table constraints of the files @@ -221,6 +389,32 @@ impl FileScanConfig { self } + /// Set the file compression type + pub fn with_file_compression_type( + mut self, + file_compression_type: FileCompressionType, + ) -> Self { + self.file_compression_type = file_compression_type; + self + } + + /// Set the new_lines_in_values property + pub fn with_newlines_in_values(mut self, new_lines_in_values: bool) -> Self { + self.new_lines_in_values = new_lines_in_values; + self + } + + /// Specifies whether newlines in (quoted) values are supported. + /// + /// Parsing newlines in quoted values may be affected by execution behaviour such as + /// parallel file scanning. Setting this to `true` ensures that newlines in values are + /// parsed successfully, which may reduce performance. + /// + /// The default behaviour depends on the `datafusion.catalog.newlines_in_values` setting. + pub fn newlines_in_values(&self) -> bool { + self.new_lines_in_values + } + /// Project the schema, constraints, and the statistics on the given column indices pub fn project(&self) -> (SchemaRef, Constraints, Statistics, Vec) { if self.projection.is_none() && self.table_partition_cols.is_empty() { @@ -387,6 +581,29 @@ impl FileScanConfig { }) .collect()) } + + // TODO: This function should be moved into DataSourceExec once FileScanConfig moved out of datafusion/core + /// Returns a new [`DataSourceExec`] from file configurations + pub fn new_exec(&self) -> Arc { + Arc::new(DataSourceExec::new(Arc::new(self.clone()))) + } + + /// Write the data_type based on file_source + fn fmt_file_source(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + write!(f, ", file_type={}", self.source.file_type())?; + self.source.fmt_extra(t, f) + } + + /// Returns the file_source + pub fn file_source(&self) -> &Arc { + &self.source + } + + fn supports_repartition(&self) -> bool { + !(self.file_compression_type.is_compressed() + || self.new_lines_in_values + || self.source.as_any().downcast_ref::().is_some()) + } } /// A helper that projects partition columns into the file record batches. @@ -642,6 +859,7 @@ mod tests { use arrow_array::Int32Array; use super::*; + use crate::datasource::physical_plan::ArrowSource; use crate::{test::columns, test_util::aggr_test_schema}; #[test] @@ -1243,10 +1461,14 @@ mod tests { statistics: Statistics, table_partition_cols: Vec, ) -> FileScanConfig { - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), file_schema) - .with_projection(projection) - .with_statistics(statistics) - .with_table_partition_cols(table_partition_cols) + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + file_schema, + Arc::new(ArrowSource::default()), + ) + .with_projection(projection) + .with_statistics(statistics) + .with_table_partition_cols(table_partition_cols) } /// Convert partition columns from Vec to Vec diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 18cda4524ab2..85b1d714548d 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -68,14 +68,14 @@ impl Default for OnError { /// stream of [`RecordBatch`] /// /// [`ObjectStore`]: object_store::ObjectStore -pub trait FileOpener: Unpin { +pub trait FileOpener: Unpin + Send + Sync { /// Asynchronously open the specified file and return a stream /// of [`RecordBatch`] fn open(&self, file_meta: FileMeta) -> Result; } /// A stream that iterates record batch by record batch, file over file. -pub struct FileStream { +pub struct FileStream { /// An iterator over input files. file_iter: VecDeque, /// The stream schema (file schema including partition columns and after @@ -83,9 +83,9 @@ pub struct FileStream { projected_schema: SchemaRef, /// The remaining number of records to parse, None if no limit remain: Option, - /// A generic [`FileOpener`]. Calling `open()` returns a [`FileOpenFuture`], + /// A dynamic [`FileOpener`]. Calling `open()` returns a [`FileOpenFuture`], /// which can be resolved to a stream of `RecordBatch`. - file_opener: F, + file_opener: Arc, /// The partition column projector pc_projector: PartitionColumnProjector, /// The stream state @@ -177,7 +177,7 @@ struct FileStreamMetrics { /// Time between when the [`FileStream`] requests data from the /// stream and when the first [`RecordBatch`] is produced. pub time_scanning_until_data: StartableTime, - /// Total elapsed wall clock time for for scanning + record batch decompression / decoding + /// Total elapsed wall clock time for scanning + record batch decompression / decoding /// /// Sum of time between when the [`FileStream`] requests data from /// the stream and when a [`RecordBatch`] is produced for all @@ -243,12 +243,12 @@ impl FileStreamMetrics { } } -impl FileStream { +impl FileStream { /// Create a new `FileStream` using the give `FileOpener` to scan underlying files pub fn new( config: &FileScanConfig, partition: usize, - file_opener: F, + file_opener: Arc, metrics: &ExecutionPlanMetricsSet, ) -> Result { let (projected_schema, ..) = config.project(); @@ -495,7 +495,7 @@ impl FileStream { } } -impl Stream for FileStream { +impl Stream for FileStream { type Item = Result; fn poll_next( @@ -509,7 +509,7 @@ impl Stream for FileStream { } } -impl RecordBatchStream for FileStream { +impl RecordBatchStream for FileStream { fn schema(&self) -> SchemaRef { Arc::clone(&self.projected_schema) } @@ -525,6 +525,7 @@ mod tests { use crate::prelude::SessionContext; use crate::test::{make_partition, object_store::register_test_store}; + use crate::datasource::physical_plan::CsvSource; use arrow_schema::Schema; use datafusion_common::internal_err; @@ -648,13 +649,15 @@ mod tests { let config = FileScanConfig::new( ObjectStoreUrl::parse("test:///").unwrap(), file_schema, + Arc::new(CsvSource::default()), ) .with_file_group(file_group) .with_limit(self.limit); let metrics_set = ExecutionPlanMetricsSet::new(); - let file_stream = FileStream::new(&config, 0, self.opener, &metrics_set) - .unwrap() - .with_on_error(on_error); + let file_stream = + FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set) + .unwrap() + .with_on_error(on_error); file_stream .collect::>() diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 7ac062e549c4..f581f5cd0b4a 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -22,27 +22,26 @@ use std::io::{BufReader, Read, Seek, SeekFrom}; use std::sync::Arc; use std::task::Poll; -use super::{calculate_range, FileGroupPartitioner, FileScanConfig, RangeCalculation}; +use super::{calculate_range, FileScanConfig, RangeCalculation}; +use crate::datasource::data_source::FileSource; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; -use crate::datasource::physical_plan::file_stream::{ - FileOpenFuture, FileOpener, FileStream, -}; +use crate::datasource::physical_plan::file_stream::{FileOpenFuture, FileOpener}; use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; -use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, Partitioning, - PlanProperties, SendableRecordBatchStream, Statistics, -}; +use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; -use datafusion_common::Constraints; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_common::{Constraints, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::source::DataSourceExec; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; @@ -52,15 +51,14 @@ use tokio::task::JoinSet; /// Execution plan for scanning NdJson data source #[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] pub struct NdJsonExec { + inner: DataSourceExec, base_config: FileScanConfig, - projected_statistics: Statistics, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, file_compression_type: FileCompressionType, - cache: PlanProperties, } +#[allow(unused, deprecated)] impl NdJsonExec { /// Create a new JSON reader execution plan provided base configurations pub fn new( @@ -79,12 +77,16 @@ impl NdJsonExec { projected_constraints, &base_config, ); + + let json = JsonSource::default(); + let base_config = base_config + .with_file_compression_type(file_compression_type) + .with_source(Arc::new(json)); + Self { + inner: DataSourceExec::new(Arc::new(base_config.clone())), + file_compression_type: base_config.file_compression_type, base_config, - projected_statistics, - metrics: ExecutionPlanMetricsSet::new(), - file_compression_type, - cache, } } @@ -98,6 +100,25 @@ impl NdJsonExec { &self.file_compression_type } + fn file_scan_config(&self) -> FileScanConfig { + let source = self.inner.source(); + source + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + fn json_source(&self) -> JsonSource { + let source = self.file_scan_config(); + source + .file_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) } @@ -122,25 +143,26 @@ impl NdJsonExec { } fn with_file_groups(mut self, file_groups: Vec>) -> Self { - self.base_config.file_groups = file_groups; - // Changing file groups may invalidate output partitioning. Update it also - let output_partitioning = Self::output_partitioning_helper(&self.base_config); - self.cache = self.cache.with_partitioning(output_partitioning); + self.base_config.file_groups = file_groups.clone(); + let mut file_source = self.file_scan_config(); + file_source = file_source.with_file_groups(file_groups); + self.inner = self.inner.with_source(Arc::new(file_source)); self } } +#[allow(unused, deprecated)] impl DisplayAs for NdJsonExec { fn fmt_as( &self, t: DisplayFormatType, f: &mut std::fmt::Formatter, ) -> std::fmt::Result { - write!(f, "JsonExec: ")?; - self.base_config.fmt_as(t, f) + self.inner.fmt_as(t, f) } } +#[allow(unused, deprecated)] impl ExecutionPlan for NdJsonExec { fn name(&self) -> &'static str { "NdJsonExec" @@ -150,7 +172,7 @@ impl ExecutionPlan for NdJsonExec { self } fn properties(&self) -> &PlanProperties { - &self.cache + self.inner.properties() } fn children(&self) -> Vec<&Arc> { @@ -169,26 +191,7 @@ impl ExecutionPlan for NdJsonExec { target_partitions: usize, config: &datafusion_common::config::ConfigOptions, ) -> Result>> { - if self.file_compression_type.is_compressed() { - return Ok(None); - } - let repartition_file_min_size = config.optimizer.repartition_file_min_size; - let preserve_order_within_groups = self.properties().output_ordering().is_some(); - let file_groups = &self.base_config.file_groups; - - let repartitioned_file_groups_option = FileGroupPartitioner::new() - .with_target_partitions(target_partitions) - .with_preserve_order_within_groups(preserve_order_within_groups) - .with_repartition_file_min_size(repartition_file_min_size) - .repartition_file_groups(file_groups); - - if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - let mut new_plan = self.clone(); - new_plan = new_plan.with_file_groups(repartitioned_file_groups); - return Ok(Some(Arc::new(new_plan))); - } - - Ok(None) + self.inner.repartitioned(target_partitions, config) } fn execute( @@ -196,46 +199,23 @@ impl ExecutionPlan for NdJsonExec { partition: usize, context: Arc, ) -> Result { - let batch_size = context.session_config().batch_size(); - - let object_store = context - .runtime_env() - .object_store(&self.base_config.object_store_url)?; - let opener = JsonOpener { - batch_size, - projected_schema: self.base_config.projected_file_schema(), - file_compression_type: self.file_compression_type.to_owned(), - object_store, - }; - - let stream = - FileStream::new(&self.base_config, partition, opener, &self.metrics)?; - - Ok(Box::pin(stream) as SendableRecordBatchStream) + self.inner.execute(partition, context) } fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) + self.inner.statistics() } fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + self.inner.metrics() } fn fetch(&self) -> Option { - self.base_config.limit + self.inner.fetch() } fn with_fetch(&self, limit: Option) -> Option> { - let new_config = self.base_config.clone().with_limit(limit); - - Some(Arc::new(Self { - base_config: new_config, - projected_statistics: self.projected_statistics.clone(), - metrics: self.metrics.clone(), - file_compression_type: self.file_compression_type, - cache: self.cache.clone(), - })) + self.inner.with_fetch(limit) } } @@ -264,6 +244,77 @@ impl JsonOpener { } } +/// JsonSource holds the extra configuration that is necessary for [`JsonOpener`] +#[derive(Clone, Default)] +pub struct JsonSource { + batch_size: Option, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, +} + +impl JsonSource { + /// Initialize a JsonSource with default values + pub fn new() -> Self { + Self::default() + } +} + +impl FileSource for JsonSource { + fn create_file_opener( + &self, + object_store: Result>, + base_config: &FileScanConfig, + _partition: usize, + ) -> Result> { + Ok(Arc::new(JsonOpener { + batch_size: self + .batch_size + .expect("Batch size must set before creating opener"), + projected_schema: base_config.projected_file_schema(), + file_compression_type: base_config.file_compression_type, + object_store: object_store?, + })) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(Self { ..self.clone() }) + } + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut conf = self.clone(); + conf.projected_statistics = Some(statistics); + Arc::new(conf) + } + + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn statistics(&self) -> Result { + let statistics = &self.projected_statistics; + Ok(statistics + .clone() + .expect("projected_statistics must be set to call")) + } + + fn file_type(&self) -> &str { + "json" + } +} + impl FileOpener for JsonOpener { /// Open a partitioned NDJSON file. /// @@ -394,6 +445,7 @@ mod tests { use super::*; use crate::dataframe::DataFrameWriteOptions; use crate::datasource::file_format::{json::JsonFormat, FileFormat}; + use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; use crate::execution::context::SessionState; use crate::prelude::{ @@ -405,6 +457,7 @@ mod tests { use arrow::array::Array; use arrow::datatypes::{Field, SchemaBuilder}; use datafusion_common::cast::{as_int32_array, as_int64_array, as_string_array}; + use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; @@ -527,12 +580,12 @@ mod tests { let (object_store_url, file_groups, file_schema) = prepare_store(&state, file_compression_type.to_owned(), tmp_dir.path()).await; - let exec = NdJsonExec::new( - FileScanConfig::new(object_store_url, file_schema) - .with_file_groups(file_groups) - .with_limit(Some(3)), - file_compression_type.to_owned(), - ); + let source = Arc::new(JsonSource::new()); + let conf = FileScanConfig::new(object_store_url, file_schema, source) + .with_file_groups(file_groups) + .with_limit(Some(3)) + .with_file_compression_type(file_compression_type.to_owned()); + let exec = conf.new_exec(); // TODO: this is not where schema inference should be tested @@ -598,12 +651,12 @@ mod tests { let file_schema = Arc::new(builder.finish()); let missing_field_idx = file_schema.fields.len() - 1; - let exec = NdJsonExec::new( - FileScanConfig::new(object_store_url, file_schema) - .with_file_groups(file_groups) - .with_limit(Some(3)), - file_compression_type.to_owned(), - ); + let source = Arc::new(JsonSource::new()); + let conf = FileScanConfig::new(object_store_url, file_schema, source) + .with_file_groups(file_groups) + .with_limit(Some(3)) + .with_file_compression_type(file_compression_type.to_owned()); + let exec = conf.new_exec(); let mut it = exec.execute(0, task_ctx)?; let batch = it.next().await.unwrap()?; @@ -638,12 +691,12 @@ mod tests { let (object_store_url, file_groups, file_schema) = prepare_store(&state, file_compression_type.to_owned(), tmp_dir.path()).await; - let exec = NdJsonExec::new( - FileScanConfig::new(object_store_url, file_schema) - .with_file_groups(file_groups) - .with_projection(Some(vec![0, 2])), - file_compression_type.to_owned(), - ); + let source = Arc::new(JsonSource::new()); + let conf = FileScanConfig::new(object_store_url, file_schema, source) + .with_file_groups(file_groups) + .with_projection(Some(vec![0, 2])) + .with_file_compression_type(file_compression_type.to_owned()); + let exec = conf.new_exec(); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 2); @@ -683,12 +736,12 @@ mod tests { let (object_store_url, file_groups, file_schema) = prepare_store(&state, file_compression_type.to_owned(), tmp_dir.path()).await; - let exec = NdJsonExec::new( - FileScanConfig::new(object_store_url, file_schema) - .with_file_groups(file_groups) - .with_projection(Some(vec![3, 0, 2])), - file_compression_type.to_owned(), - ); + let source = Arc::new(JsonSource::new()); + let conf = FileScanConfig::new(object_store_url, file_schema, source) + .with_file_groups(file_groups) + .with_projection(Some(vec![3, 0, 2])) + .with_file_compression_type(file_compression_type.to_owned()); + let exec = conf.new_exec(); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 3); diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 5bb7da8376a2..28ac73cef521 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -31,18 +31,31 @@ mod statistics; pub(crate) use self::csv::plan_to_csv; pub(crate) use self::json::plan_to_json; #[cfg(feature = "parquet")] -pub use self::parquet::{ParquetExec, ParquetFileMetrics, ParquetFileReaderFactory}; +pub use self::parquet::source::ParquetSource; +#[cfg(feature = "parquet")] +#[allow(deprecated)] +pub use self::parquet::{ + ParquetExec, ParquetExecBuilder, ParquetFileMetrics, ParquetFileReaderFactory, +}; +#[allow(deprecated)] pub use arrow_file::ArrowExec; +pub use arrow_file::ArrowSource; +#[allow(deprecated)] pub use avro::AvroExec; -pub use csv::{CsvConfig, CsvExec, CsvExecBuilder, CsvOpener}; +pub use avro::AvroSource; +#[allow(deprecated)] +pub use csv::{CsvExec, CsvExecBuilder}; +pub use csv::{CsvOpener, CsvSource}; use datafusion_expr::dml::InsertOp; pub use file_groups::FileGroupPartitioner; pub use file_scan_config::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, }; pub use file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; -pub use json::{JsonOpener, NdJsonExec}; +#[allow(deprecated)] +pub use json::NdJsonExec; +pub use json::{JsonOpener, JsonSource}; use std::{ fmt::{Debug, Formatter, Result as FmtResult}, @@ -367,7 +380,7 @@ impl From for FileMeta { ///┃ Partition 1 Partition 2 Partition 3 Partition 4 ┃ /// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ /// -/// ParquetExec +/// DataSourceExec ///``` /// /// However, when more than 1 file is assigned to each partition, each @@ -393,7 +406,7 @@ impl From for FileMeta { ///┃ Partition 1 Partition 2 /// ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ━ ┛ /// -/// ParquetExec +/// DataSourceExec ///``` fn get_projected_output_ordering( base_config: &FileScanConfig, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs b/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs index 0d77a99699bd..d30549708bbd 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs @@ -21,7 +21,7 @@ use parquet::file::metadata::RowGroupMetaData; /// A selection of rows and row groups within a ParquetFile to decode. /// -/// A `ParquetAccessPlan` is used to limit the row groups and data pages a `ParquetExec` +/// A `ParquetAccessPlan` is used to limit the row groups and data pages a `DataSourceExec` /// will read and decode to improve performance. /// /// Note that page level pruning based on ArrowPredicate is applied after all of diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 2c867120b3a0..6b1cb23ac1f7 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! [`ParquetExec`] Execution plan for reading Parquet files +//! [`ParquetExec`] FileSource for reading Parquet files mod access_plan; mod metrics; @@ -24,275 +24,79 @@ mod page_filter; mod reader; mod row_filter; mod row_group_filter; +pub mod source; mod writer; use std::any::Any; -use std::fmt::Debug; +use std::fmt::Formatter; use std::sync::Arc; use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::file_stream::FileStream; -use crate::datasource::physical_plan::{DisplayAs, FileGroupPartitioner, FileScanConfig}; -use crate::datasource::schema_adapter::{ - DefaultSchemaAdapterFactory, SchemaAdapterFactory, +use crate::datasource::physical_plan::{ + parquet::source::ParquetSource, DisplayAs, FileScanConfig, }; +use crate::datasource::schema_adapter::SchemaAdapterFactory; use crate::{ - config::{ConfigOptions, TableParquetOptions}, + config::TableParquetOptions, error::Result, execution::context::TaskContext, physical_plan::{ - metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, Statistics, + metrics::MetricsSet, DisplayFormatType, ExecutionPlan, Partitioning, + PlanProperties, SendableRecordBatchStream, Statistics, }, }; pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; use arrow::datatypes::SchemaRef; +use datafusion_common::config::ConfigOptions; use datafusion_common::Constraints; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; use datafusion_physical_optimizer::pruning::PruningPredicate; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::source::DataSourceExec; pub use metrics::ParquetFileMetrics; -use opener::ParquetOpener; pub use page_filter::PagePruningAccessPlanFilter; pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; pub use row_filter::can_expr_be_pushed_down_with_schemas; pub use row_group_filter::RowGroupAccessPlanFilter; pub use writer::plan_to_parquet; -use itertools::Itertools; use log::debug; -/// Execution plan for reading one or more Parquet files. -/// -/// ```text -/// ▲ -/// │ -/// │ Produce a stream of -/// │ RecordBatches -/// │ -/// ┌───────────────────────┐ -/// │ │ -/// │ ParquetExec │ -/// │ │ -/// └───────────────────────┘ -/// ▲ -/// │ Asynchronously read from one -/// │ or more parquet files via -/// │ ObjectStore interface -/// │ -/// │ -/// .───────────────────. -/// │ ) -/// │`───────────────────'│ -/// │ ObjectStore │ -/// │.───────────────────.│ -/// │ ) -/// `───────────────────' -/// -/// ``` -/// -/// # Example: Create a `ParquetExec` -/// ``` -/// # use std::sync::Arc; -/// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; -/// # use datafusion::datasource::listing::PartitionedFile; -/// # let file_schema = Arc::new(Schema::empty()); -/// # let object_store_url = ObjectStoreUrl::local_filesystem(); -/// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion_physical_expr::expressions::lit; -/// # let predicate = lit(true); -/// // Create a ParquetExec for reading `file1.parquet` with a file size of 100MB -/// let file_scan_config = FileScanConfig::new(object_store_url, file_schema) -/// .with_file(PartitionedFile::new("file1.parquet", 100*1024*1024)); -/// let exec = ParquetExec::builder(file_scan_config) -/// // Provide a predicate for filtering row groups/pages -/// .with_predicate(predicate) -/// .build(); -/// ``` -/// -/// # Features -/// -/// Supports the following optimizations: -/// -/// * Concurrent reads: reads from one or more files in parallel as multiple -/// partitions, including concurrently reading multiple row groups from a single -/// file. -/// -/// * Predicate push down: skips row groups, pages, rows based on metadata -/// and late materialization. See "Predicate Pushdown" below. -/// -/// * Projection pushdown: reads and decodes only the columns required. -/// -/// * Limit pushdown: stop execution early after some number of rows are read. -/// -/// * Custom readers: customize reading parquet files, e.g. to cache metadata, -/// coalesce I/O operations, etc. See [`ParquetFileReaderFactory`] for more -/// details. -/// -/// * Schema evolution: read parquet files with different schemas into a unified -/// table schema. See [`SchemaAdapterFactory`] for more details. -/// -/// * metadata_size_hint: controls the number of bytes read from the end of the -/// file in the initial I/O when the default [`ParquetFileReaderFactory`]. If a -/// custom reader is used, it supplies the metadata directly and this parameter -/// is ignored. [`ParquetExecBuilder::with_metadata_size_hint`] for more details. -/// -/// * User provided [`ParquetAccessPlan`]s to skip row groups and/or pages -/// based on external information. See "Implementing External Indexes" below -/// -/// # Predicate Pushdown -/// -/// `ParquetExec` uses the provided [`PhysicalExpr`] predicate as a filter to -/// skip reading unnecessary data and improve query performance using several techniques: -/// -/// * Row group pruning: skips entire row groups based on min/max statistics -/// found in [`ParquetMetaData`] and any Bloom filters that are present. -/// -/// * Page pruning: skips individual pages within a ColumnChunk using the -/// [Parquet PageIndex], if present. -/// -/// * Row filtering: skips rows within a page using a form of late -/// materialization. When possible, predicates are applied by the parquet -/// decoder *during* decode (see [`ArrowPredicate`] and [`RowFilter`] for more -/// details). This is only enabled if `ParquetScanOptions::pushdown_filters` is set to true. -/// -/// Note: If the predicate can not be used to accelerate the scan, it is ignored -/// (no error is raised on predicate evaluation errors). -/// -/// [`ArrowPredicate`]: parquet::arrow::arrow_reader::ArrowPredicate -/// [`RowFilter`]: parquet::arrow::arrow_reader::RowFilter -/// [Parquet PageIndex]: https://github.com/apache/parquet-format/blob/master/PageIndex.md -/// -/// # Example: rewriting `ParquetExec` -/// -/// You can modify a `ParquetExec` using [`ParquetExecBuilder`], for example -/// to change files or add a predicate. -/// -/// ```no_run -/// # use std::sync::Arc; -/// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; -/// # use datafusion::datasource::listing::PartitionedFile; -/// # fn parquet_exec() -> ParquetExec { unimplemented!() } -/// // Split a single ParquetExec into multiple ParquetExecs, one for each file -/// let exec = parquet_exec(); -/// let existing_file_groups = &exec.base_config().file_groups; -/// let new_execs = existing_file_groups -/// .iter() -/// .map(|file_group| { -/// // create a new exec by copying the existing exec into a builder -/// let new_exec = exec.clone() -/// .into_builder() -/// .with_file_groups(vec![file_group.clone()]) -/// .build(); -/// new_exec -/// }) -/// .collect::>(); -/// ``` -/// -/// # Implementing External Indexes -/// -/// It is possible to restrict the row groups and selections within those row -/// groups that the ParquetExec will consider by providing an initial -/// [`ParquetAccessPlan`] as `extensions` on [`PartitionedFile`]. This can be -/// used to implement external indexes on top of parquet files and select only -/// portions of the files. -/// -/// The `ParquetExec` will try and reduce any provided `ParquetAccessPlan` -/// further based on the contents of `ParquetMetadata` and other settings. -/// -/// ## Example of providing a ParquetAccessPlan -/// -/// ``` -/// # use std::sync::Arc; -/// # use arrow_schema::{Schema, SchemaRef}; -/// # use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; -/// # use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; -/// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # fn schema() -> SchemaRef { -/// # Arc::new(Schema::empty()) -/// # } -/// // create an access plan to scan row group 0, 1 and 3 and skip row groups 2 and 4 -/// let mut access_plan = ParquetAccessPlan::new_all(5); -/// access_plan.skip(2); -/// access_plan.skip(4); -/// // provide the plan as extension to the FileScanConfig -/// let partitioned_file = PartitionedFile::new("my_file.parquet", 1234) -/// .with_extensions(Arc::new(access_plan)); -/// // create a ParquetExec to scan this file -/// let file_scan_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema()) -/// .with_file(partitioned_file); -/// // this parquet exec will not even try to read row groups 2 and 4. Additional -/// // pruning based on predicates may also happen -/// let exec = ParquetExec::builder(file_scan_config).build(); -/// ``` -/// -/// For a complete example, see the [`advanced_parquet_index` example]). -/// -/// [`parquet_index_advanced` example]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs -/// -/// # Execution Overview -/// -/// * Step 1: [`ParquetExec::execute`] is called, returning a [`FileStream`] -/// configured to open parquet files with a `ParquetOpener`. -/// -/// * Step 2: When the stream is polled, the `ParquetOpener` is called to open -/// the file. -/// -/// * Step 3: The `ParquetOpener` gets the [`ParquetMetaData`] (file metadata) -/// via [`ParquetFileReaderFactory`], creating a [`ParquetAccessPlan`] by -/// applying predicates to metadata. The plan and projections are used to -/// determine what pages must be read. -/// -/// * Step 4: The stream begins reading data, fetching the required parquet -/// pages incrementally decoding them, and applying any row filters (see -/// [`Self::with_pushdown_filters`]). -/// -/// * Step 5: As each [`RecordBatch`] is read, it may be adapted by a -/// [`SchemaAdapter`] to match the table schema. By default missing columns are -/// filled with nulls, but this can be customized via [`SchemaAdapterFactory`]. -/// -/// [`RecordBatch`]: arrow::record_batch::RecordBatch -/// [`SchemaAdapter`]: crate::datasource::schema_adapter::SchemaAdapter -/// [`ParquetMetadata`]: parquet::file::metadata::ParquetMetaData #[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] +/// Deprecated Execution plan replaced with DataSourceExec pub struct ParquetExec { - /// Base configuration for this scan + inner: DataSourceExec, base_config: FileScanConfig, - projected_statistics: Statistics, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, + table_parquet_options: TableParquetOptions, /// Optional predicate for row filtering during parquet scan predicate: Option>, /// Optional predicate for pruning row groups (derived from `predicate`) pruning_predicate: Option>, - /// Optional predicate for pruning pages (derived from `predicate`) - page_pruning_predicate: Option>, - /// Optional hint for the size of the parquet metadata - metadata_size_hint: Option, /// Optional user defined parquet file reader factory parquet_file_reader_factory: Option>, - /// Cached plan properties such as equivalence properties, ordering, partitioning, etc. - cache: PlanProperties, - /// Options for reading Parquet files - table_parquet_options: TableParquetOptions, /// Optional user defined schema adapter schema_adapter_factory: Option>, } +#[allow(unused, deprecated)] impl From for ParquetExecBuilder { fn from(exec: ParquetExec) -> Self { exec.into_builder() } } -/// [`ParquetExecBuilder`], builder for [`ParquetExec`]. +/// [`ParquetExecBuilder`], deprecated builder for [`ParquetExec`]. /// -/// See example on [`ParquetExec`]. +/// ParquetExec is replaced with `DataSourceExec` and it includes `ParquetSource` +/// +/// See example on [`ParquetSource`]. +#[deprecated( + since = "46.0.0", + note = "use DataSourceExec with ParquetSource instead" +)] +#[allow(unused, deprecated)] pub struct ParquetExecBuilder { file_scan_config: FileScanConfig, predicate: Option>, @@ -302,6 +106,7 @@ pub struct ParquetExecBuilder { schema_adapter_factory: Option>, } +#[allow(unused, deprecated)] impl ParquetExecBuilder { /// Create a new builder to read the provided file scan configuration pub fn new(file_scan_config: FileScanConfig) -> Self { @@ -412,72 +217,40 @@ impl ParquetExecBuilder { parquet_file_reader_factory, schema_adapter_factory, } = self; + let mut parquet = ParquetSource::new(table_parquet_options); + if let Some(predicate) = predicate.clone() { + parquet = parquet + .with_predicate(Arc::clone(&file_scan_config.file_schema), predicate); + } + if let Some(metadata_size_hint) = metadata_size_hint { + parquet = parquet.with_metadata_size_hint(metadata_size_hint) + } + if let Some(parquet_reader_factory) = parquet_file_reader_factory { + parquet = parquet.with_parquet_file_reader_factory(parquet_reader_factory) + } + if let Some(schema_factory) = schema_adapter_factory { + parquet = parquet.with_schema_adapter_factory(schema_factory); + } - let base_config = file_scan_config; + let base_config = file_scan_config.with_source(Arc::new(parquet.clone())); debug!("Creating ParquetExec, files: {:?}, 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 file_schema = &base_config.file_schema; - let pruning_predicate = predicate - .clone() - .and_then(|predicate_expr| { - match PruningPredicate::try_new(predicate_expr, Arc::clone(file_schema)) { - Ok(pruning_predicate) => Some(Arc::new(pruning_predicate)), - Err(e) => { - debug!("Could not create pruning predicate for: {e}"); - predicate_creation_errors.add(1); - None - } - } - }) - .filter(|p| !p.always_true()); - - let page_pruning_predicate = predicate - .as_ref() - .map(|predicate_expr| { - PagePruningAccessPlanFilter::new(predicate_expr, Arc::clone(file_schema)) - }) - .map(Arc::new); - - let ( - projected_schema, - projected_constraints, - projected_statistics, - projected_output_ordering, - ) = base_config.project(); - - let cache = ParquetExec::compute_properties( - projected_schema, - &projected_output_ordering, - projected_constraints, - &base_config, - ); ParquetExec { + inner: DataSourceExec::new(Arc::new(base_config.clone())), base_config, - projected_statistics, - metrics, predicate, - pruning_predicate, - page_pruning_predicate, - metadata_size_hint, - parquet_file_reader_factory, - cache, - table_parquet_options, - schema_adapter_factory, + pruning_predicate: parquet.pruning_predicate, + schema_adapter_factory: parquet.schema_adapter_factory, + parquet_file_reader_factory: parquet.parquet_file_reader_factory, + table_parquet_options: parquet.table_parquet_options, } } } +#[allow(unused, deprecated)] impl ParquetExec { /// Create a new Parquet reader execution plan provided file list and schema. - #[deprecated( - since = "39.0.0", - note = "use `ParquetExec::builder` or `ParquetExecBuilder`" - )] pub fn new( base_config: FileScanConfig, predicate: Option>, @@ -494,7 +267,6 @@ impl ParquetExec { } builder.build() } - /// Return a [`ParquetExecBuilder`]. /// /// See example on [`ParquetExec`] and [`ParquetExecBuilder`] for specifying @@ -508,96 +280,119 @@ impl ParquetExec { // list out fields so it is clear what is being dropped // (note the fields which are dropped are re-created as part of calling // `build` on the builder) - let Self { - base_config, - projected_statistics: _, - metrics: _, - predicate, - pruning_predicate: _, - page_pruning_predicate: _, - metadata_size_hint, - parquet_file_reader_factory, - cache: _, - table_parquet_options, - schema_adapter_factory, - } = self; + let file_scan_config = self.file_scan_config(); + let parquet = self.parquet_source(); + ParquetExecBuilder { - file_scan_config: base_config, - predicate, - metadata_size_hint, - table_parquet_options, - parquet_file_reader_factory, - schema_adapter_factory, + file_scan_config, + predicate: parquet.predicate, + metadata_size_hint: parquet.metadata_size_hint, + table_parquet_options: parquet.table_parquet_options, + parquet_file_reader_factory: parquet.parquet_file_reader_factory, + schema_adapter_factory: parquet.schema_adapter_factory, } } + fn file_scan_config(&self) -> FileScanConfig { + let source = self.inner.source(); + source + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + fn parquet_source(&self) -> ParquetSource { + let source = self.file_scan_config(); + source + .file_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } /// [`FileScanConfig`] that controls this scan (such as which files to read) pub fn base_config(&self) -> &FileScanConfig { &self.base_config } - /// Options passed to the parquet reader for this scan pub fn table_parquet_options(&self) -> &TableParquetOptions { &self.table_parquet_options } - /// Optional predicate. pub fn predicate(&self) -> Option<&Arc> { self.predicate.as_ref() } - /// Optional reference to this parquet scan's pruning predicate pub fn pruning_predicate(&self) -> Option<&Arc> { self.pruning_predicate.as_ref() } - /// return the optional file reader factory pub fn parquet_file_reader_factory( &self, ) -> Option<&Arc> { self.parquet_file_reader_factory.as_ref() } - /// Optional user defined parquet file reader factory. - /// pub fn with_parquet_file_reader_factory( mut self, parquet_file_reader_factory: Arc, ) -> Self { + let mut parquet = self.parquet_source(); + parquet.parquet_file_reader_factory = + Some(Arc::clone(&parquet_file_reader_factory)); + let file_source = self.file_scan_config(); + self.inner = self + .inner + .with_source(Arc::new(file_source.with_source(Arc::new(parquet)))); self.parquet_file_reader_factory = Some(parquet_file_reader_factory); self } - /// return the optional schema adapter factory pub fn schema_adapter_factory(&self) -> Option<&Arc> { self.schema_adapter_factory.as_ref() } - - /// Optional schema adapter factory. + /// Set optional schema adapter factory. /// - /// See documentation on [`ParquetExecBuilder::with_schema_adapter_factory`] + /// [`SchemaAdapterFactory`] allows user to specify how fields from the + /// parquet file get mapped to that of the table schema. The default schema + /// adapter uses arrow's cast library to map the parquet fields to the table + /// schema. pub fn with_schema_adapter_factory( mut self, schema_adapter_factory: Arc, ) -> Self { + let mut parquet = self.parquet_source(); + parquet.schema_adapter_factory = Some(Arc::clone(&schema_adapter_factory)); + let file_source = self.file_scan_config(); + self.inner = self + .inner + .with_source(Arc::new(file_source.with_source(Arc::new(parquet)))); self.schema_adapter_factory = Some(schema_adapter_factory); self } - /// If true, the predicate will be used during the parquet scan. /// Defaults to false /// /// [`Expr`]: datafusion_expr::Expr pub fn with_pushdown_filters(mut self, pushdown_filters: bool) -> Self { + let mut parquet = self.parquet_source(); + parquet.table_parquet_options.global.pushdown_filters = pushdown_filters; + let file_source = self.file_scan_config(); + self.inner = self + .inner + .with_source(Arc::new(file_source.with_source(Arc::new(parquet)))); self.table_parquet_options.global.pushdown_filters = pushdown_filters; self } /// Return the value described in [`Self::with_pushdown_filters`] fn pushdown_filters(&self) -> bool { - self.table_parquet_options.global.pushdown_filters + self.parquet_source() + .table_parquet_options + .global + .pushdown_filters } - /// If true, the `RowFilter` made by `pushdown_filters` may try to /// minimize the cost of filter evaluation by reordering the /// predicate [`Expr`]s. If false, the predicates are applied in @@ -605,48 +400,38 @@ impl ParquetExec { /// /// [`Expr`]: datafusion_expr::Expr pub fn with_reorder_filters(mut self, reorder_filters: bool) -> Self { + let mut parquet = self.parquet_source(); + parquet.table_parquet_options.global.reorder_filters = reorder_filters; + let file_source = self.file_scan_config(); + self.inner = self + .inner + .with_source(Arc::new(file_source.with_source(Arc::new(parquet)))); self.table_parquet_options.global.reorder_filters = reorder_filters; self } - /// Return the value described in [`Self::with_reorder_filters`] fn reorder_filters(&self) -> bool { - self.table_parquet_options.global.reorder_filters + self.parquet_source() + .table_parquet_options + .global + .reorder_filters } - /// If enabled, the reader will read the page index /// This is used to optimize filter pushdown /// via `RowSelector` and `RowFilter` by /// eliminating unnecessary IO and decoding - pub fn with_enable_page_index(mut self, enable_page_index: bool) -> Self { - self.table_parquet_options.global.enable_page_index = enable_page_index; - self + fn bloom_filter_on_read(&self) -> bool { + self.parquet_source() + .table_parquet_options + .global + .bloom_filter_on_read } - - /// Return the value described in [`Self::with_enable_page_index`] + /// Return the value described in [`ParquetSource::with_enable_page_index`] fn enable_page_index(&self) -> bool { - self.table_parquet_options.global.enable_page_index - } - - /// If enabled, the reader will read by the bloom filter - pub fn with_bloom_filter_on_read(mut self, bloom_filter_on_read: bool) -> Self { - self.table_parquet_options.global.bloom_filter_on_read = bloom_filter_on_read; - self - } - - /// If enabled, the writer will write by the bloom filter - pub fn with_bloom_filter_on_write( - mut self, - enable_bloom_filter_on_write: bool, - ) -> Self { - self.table_parquet_options.global.bloom_filter_on_write = - enable_bloom_filter_on_write; - self - } - - /// Return the value described in [`Self::with_bloom_filter_on_read`] - fn bloom_filter_on_read(&self) -> bool { - self.table_parquet_options.global.bloom_filter_on_read + self.parquet_source() + .table_parquet_options + .global + .enable_page_index } fn output_partitioning_helper(file_config: &FileScanConfig) -> Partitioning { @@ -677,54 +462,21 @@ impl ParquetExec { mut self, file_groups: Vec>, ) -> Self { - self.base_config.file_groups = file_groups; - // Changing file groups may invalidate output partitioning. Update it also - let output_partitioning = Self::output_partitioning_helper(&self.base_config); - self.cache = self.cache.with_partitioning(output_partitioning); + let mut config = self.file_scan_config(); + config.file_groups = file_groups; + self.inner = self.inner.with_source(Arc::new(config)); self } } +#[allow(unused, deprecated)] impl DisplayAs for ParquetExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - let predicate_string = self - .predicate - .as_ref() - .map(|p| format!(", predicate={p}")) - .unwrap_or_default(); - - let pruning_predicate_string = self - .pruning_predicate - .as_ref() - .map(|pre| { - let mut guarantees = pre - .literal_guarantees() - .iter() - .map(|item| format!("{}", item)) - .collect_vec(); - guarantees.sort(); - format!( - ", pruning_predicate={}, required_guarantees=[{}]", - pre.predicate_expr(), - guarantees.join(", ") - ) - }) - .unwrap_or_default(); - - write!(f, "ParquetExec: ")?; - self.base_config.fmt_as(t, f)?; - write!(f, "{}{}", predicate_string, pruning_predicate_string,) - } - } + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + self.inner.fmt_as(t, f) } } +#[allow(unused, deprecated)] impl ExecutionPlan for ParquetExec { fn name(&self) -> &'static str { "ParquetExec" @@ -736,7 +488,7 @@ impl ExecutionPlan for ParquetExec { } fn properties(&self) -> &PlanProperties { - &self.cache + self.inner.properties() } fn children(&self) -> Vec<&Arc> { @@ -752,27 +504,13 @@ impl ExecutionPlan for ParquetExec { } /// Redistribute files across partitions according to their size - /// See comments on [`FileGroupPartitioner`] for more detail. + /// See comments on `FileGroupPartitioner` for more detail. fn repartitioned( &self, target_partitions: usize, config: &ConfigOptions, ) -> Result>> { - let repartition_file_min_size = config.optimizer.repartition_file_min_size; - let repartitioned_file_groups_option = FileGroupPartitioner::new() - .with_target_partitions(target_partitions) - .with_repartition_file_min_size(repartition_file_min_size) - .with_preserve_order_within_groups( - self.properties().output_ordering().is_some(), - ) - .repartition_file_groups(&self.base_config.file_groups); - - let mut new_plan = self.clone(); - if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - new_plan = new_plan - .with_file_groups_and_update_partitioning(repartitioned_file_groups); - } - Ok(Some(Arc::new(new_plan))) + self.inner.repartitioned(target_partitions, config) } fn execute( @@ -780,94 +518,20 @@ impl ExecutionPlan for ParquetExec { partition_index: usize, ctx: Arc, ) -> Result { - let projection = self - .base_config - .file_column_projection_indices() - .unwrap_or_else(|| { - (0..self.base_config.file_schema.fields().len()).collect() - }); - - let parquet_file_reader_factory = self - .parquet_file_reader_factory - .as_ref() - .map(|f| Ok(Arc::clone(f))) - .unwrap_or_else(|| { - ctx.runtime_env() - .object_store(&self.base_config.object_store_url) - .map(|store| { - Arc::new(DefaultParquetFileReaderFactory::new(store)) as _ - }) - })?; - - let schema_adapter_factory = self - .schema_adapter_factory - .clone() - .unwrap_or_else(|| Arc::new(DefaultSchemaAdapterFactory)); - - let opener = ParquetOpener { - partition_index, - projection: Arc::from(projection), - batch_size: ctx.session_config().batch_size(), - limit: self.base_config.limit, - predicate: self.predicate.clone(), - pruning_predicate: self.pruning_predicate.clone(), - page_pruning_predicate: self.page_pruning_predicate.clone(), - table_schema: Arc::clone(&self.base_config.file_schema), - metadata_size_hint: self.metadata_size_hint, - metrics: self.metrics.clone(), - parquet_file_reader_factory, - pushdown_filters: self.pushdown_filters(), - reorder_filters: self.reorder_filters(), - enable_page_index: self.enable_page_index(), - enable_bloom_filter: self.bloom_filter_on_read(), - schema_adapter_factory, - }; - - let stream = - FileStream::new(&self.base_config, partition_index, opener, &self.metrics)?; - - Ok(Box::pin(stream)) + self.inner.execute(partition_index, ctx) } - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + self.inner.metrics() } - fn statistics(&self) -> Result { - // When filters are pushed down, we have no way of knowing the exact statistics. - // Note that pruning predicate is also a kind of filter pushdown. - // (bloom filters use `pruning_predicate` too) - let stats = if self.pruning_predicate.is_some() - || self.page_pruning_predicate.is_some() - || (self.predicate.is_some() && self.pushdown_filters()) - { - self.projected_statistics.clone().to_inexact() - } else { - self.projected_statistics.clone() - }; - Ok(stats) + self.inner.statistics() } - fn fetch(&self) -> Option { - self.base_config.limit + self.inner.fetch() } fn with_fetch(&self, limit: Option) -> Option> { - let new_config = self.base_config.clone().with_limit(limit); - - Some(Arc::new(Self { - base_config: new_config, - projected_statistics: self.projected_statistics.clone(), - metrics: self.metrics.clone(), - predicate: self.predicate.clone(), - pruning_predicate: self.pruning_predicate.clone(), - page_pruning_predicate: self.page_pruning_predicate.clone(), - metadata_size_hint: self.metadata_size_hint, - parquet_file_reader_factory: self.parquet_file_reader_factory.clone(), - cache: self.cache.clone(), - table_parquet_options: self.table_parquet_options.clone(), - schema_adapter_factory: self.schema_adapter_factory.clone(), - })) + self.inner.with_fetch(limit) } } @@ -895,7 +559,7 @@ mod tests { use crate::datasource::file_format::options::CsvReadOptions; use crate::datasource::file_format::parquet::test_util::store_parquet; use crate::datasource::file_format::test_util::scan_format; - use crate::datasource::listing::{FileRange, ListingOptions}; + use crate::datasource::listing::{FileRange, ListingOptions, PartitionedFile}; use crate::datasource::object_store::ObjectStoreUrl; use crate::execution::context::SessionState; use crate::physical_plan::displayable; @@ -918,8 +582,11 @@ mod tests { use datafusion_common::{assert_contains, ScalarValue}; use datafusion_expr::{col, lit, when, Expr}; use datafusion_physical_expr::planner::logical2physical; - use datafusion_physical_plan::ExecutionPlanProperties; + use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; + use datafusion_physical_plan::source::DataSourceExec; + use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; + use crate::datasource::physical_plan::parquet::source::ParquetSource; use chrono::{TimeZone, Utc}; use futures::StreamExt; use object_store::local::LocalFileSystem; @@ -934,7 +601,9 @@ mod tests { /// Data that was read back from ParquetFiles batches: Result>, /// The physical plan that was created (that has statistics, etc) - parquet_exec: Arc, + parquet_exec: Arc, + /// The ParquetSource that is used in plan + parquet_source: ParquetSource, } /// round-trip record batches by writing each individual RecordBatch to @@ -1015,34 +684,41 @@ mod tests { // set up predicate (this is normally done by a layer higher up) let predicate = predicate.map(|p| logical2physical(&p, &file_schema)); - // prepare the scan - let mut builder = ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) - .with_file_group(file_group) - .with_projection(projection), - ); - + let mut source = ParquetSource::default(); if let Some(predicate) = predicate { - builder = builder.with_predicate(predicate); + source = source.with_predicate(Arc::clone(&file_schema), predicate); } - let mut parquet_exec = builder.build(); if pushdown_predicate { - parquet_exec = parquet_exec + source = source .with_pushdown_filters(true) .with_reorder_filters(true); } if page_index_predicate { - parquet_exec = parquet_exec.with_enable_page_index(true); + source = source.with_enable_page_index(true); } + let base_config = FileScanConfig::new( + ObjectStoreUrl::local_filesystem(), + file_schema, + Arc::new(source.clone()), + ) + .with_file_group(file_group) + .with_projection(projection); + let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let parquet_exec = Arc::new(parquet_exec); + let parquet_exec = base_config.new_exec(); RoundTripResult { batches: collect(parquet_exec.clone(), task_ctx).await, parquet_exec, + parquet_source: base_config + .file_source() + .as_any() + .downcast_ref::() + .unwrap() + .clone(), } } } @@ -1381,7 +1057,7 @@ mod tests { // This does not look correct since the "c2" values in the result do not in fact match the predicate `c2 == 0` // but parquet pruning is not exact. If the min/max values are not defined (which they are not in this case since the it is // a null array, then the pruning predicate (currently) can not be applied. - // In a real query where this predicate was pushed down from a filter stage instead of created directly in the `ParquetExec`, + // In a real query where this predicate was pushed down from a filter stage instead of created directly in the `DataSourceExec`, // the filter stage would be preserved as a separate execution plan stage so the actual query results would be as expected. let expected = [ "+-----+----+", @@ -1673,11 +1349,13 @@ mod tests { expected_row_num: Option, file_schema: SchemaRef, ) -> Result<()> { - let parquet_exec = ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) - .with_file_groups(file_groups), + let parquet_exec = FileScanConfig::new( + ObjectStoreUrl::local_filesystem(), + file_schema, + Arc::new(ParquetSource::default()), ) - .build(); + .with_file_groups(file_groups) + .new_exec(); assert_eq!( parquet_exec .properties() @@ -1774,29 +1452,29 @@ mod tests { ), ]); - let parquet_exec = ParquetExec::builder( - FileScanConfig::new(object_store_url, schema.clone()) - .with_file(partitioned_file) - // file has 10 cols so index 12 should be month and 13 should be day - .with_projection(Some(vec![0, 1, 2, 12, 13])) - .with_table_partition_cols(vec![ - Field::new("year", DataType::Utf8, false), - Field::new("month", DataType::UInt8, false), - Field::new( - "day", - DataType::Dictionary( - Box::new(DataType::UInt16), - Box::new(DataType::Utf8), - ), - false, + let source = Arc::new(ParquetSource::default()); + let parquet_exec = FileScanConfig::new(object_store_url, schema.clone(), source) + .with_file(partitioned_file) + // file has 10 cols so index 12 should be month and 13 should be day + .with_projection(Some(vec![0, 1, 2, 12, 13])) + .with_table_partition_cols(vec![ + Field::new("year", DataType::Utf8, false), + Field::new("month", DataType::UInt8, false), + Field::new( + "day", + DataType::Dictionary( + Box::new(DataType::UInt16), + Box::new(DataType::Utf8), ), - ]), - ) - .build(); - assert_eq!( - parquet_exec.cache.output_partitioning().partition_count(), - 1 - ); + false, + ), + ]) + .new_exec(); + let partition_count = parquet_exec + .source() + .output_partitioning() + .partition_count(); + assert_eq!(partition_count, 1); assert_eq!(parquet_exec.schema().as_ref(), &expected_schema); let mut results = parquet_exec.execute(0, task_ctx)?; @@ -1848,11 +1526,13 @@ mod tests { }; let file_schema = Arc::new(Schema::empty()); - let parquet_exec = ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) - .with_file(partitioned_file), + let parquet_exec = FileScanConfig::new( + ObjectStoreUrl::local_filesystem(), + file_schema, + Arc::new(ParquetSource::default()), ) - .build(); + .with_file(partitioned_file) + .new_exec(); let mut results = parquet_exec.execute(0, state.task_ctx())?; let batch = results.next().await.unwrap(); @@ -1997,7 +1677,7 @@ mod tests { .await; // should have a pruning predicate - let pruning_predicate = &rt.parquet_exec.pruning_predicate; + let pruning_predicate = &rt.parquet_source.pruning_predicate; assert!(pruning_predicate.is_some()); // convert to explain plan form @@ -2038,7 +1718,7 @@ mod tests { .round_trip(vec![batches.clone()]) .await; - let pruning_predicate = &rt0.parquet_exec.pruning_predicate; + let pruning_predicate = &rt0.parquet_source.pruning_predicate; assert!(pruning_predicate.is_some()); let display0 = displayable(rt0.parquet_exec.as_ref()) @@ -2080,9 +1760,9 @@ mod tests { .await; // should have a pruning predicate - let pruning_predicate = &rt1.parquet_exec.pruning_predicate; + let pruning_predicate = &rt1.parquet_source.pruning_predicate; assert!(pruning_predicate.is_some()); - let pruning_predicate = &rt2.parquet_exec.pruning_predicate; + let pruning_predicate = &rt2.parquet_source.predicate; assert!(pruning_predicate.is_some()); // convert to explain plan form @@ -2123,14 +1803,14 @@ mod tests { .await; // Should not contain a pruning predicate (since nothing can be pruned) - let pruning_predicate = &rt.parquet_exec.pruning_predicate; + let pruning_predicate = &rt.parquet_source.pruning_predicate; assert!( pruning_predicate.is_none(), "Still had pruning predicate: {pruning_predicate:?}" ); // but does still has a pushdown down predicate - let predicate = rt.parquet_exec.predicate.as_ref(); + let predicate = rt.parquet_source.predicate.as_ref(); let filter_phys = logical2physical(&filter, rt.parquet_exec.schema().as_ref()); assert_eq!(predicate.unwrap().to_string(), filter_phys.to_string()); } @@ -2158,11 +1838,11 @@ mod tests { .await; // Should have a pruning predicate - let pruning_predicate = &rt.parquet_exec.pruning_predicate; + let pruning_predicate = &rt.parquet_source.pruning_predicate; assert!(pruning_predicate.is_some()); } - /// returns the sum of all the metrics with the specified name + /// Returns the sum of all the metrics with the specified name /// the returned set. /// /// Count: returns value @@ -2472,30 +2152,18 @@ mod tests { let size_hint_calls = reader_factory.metadata_size_hint_calls.clone(); - let exec = ParquetExec::builder( - FileScanConfig::new(store_url, schema) - .with_file( - PartitionedFile { - object_meta: ObjectMeta { - location: Path::from(name_1), - last_modified: Utc::now(), - size: total_size_1, - e_tag: None, - version: None, - }, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - } - .with_metadata_size_hint(123), - ) - .with_file(PartitionedFile { + let source = Arc::new( + ParquetSource::default() + .with_parquet_file_reader_factory(reader_factory) + .with_metadata_size_hint(456), + ); + let exec = FileScanConfig::new(store_url, schema, source) + .with_file( + PartitionedFile { object_meta: ObjectMeta { - location: Path::from(name_2), + location: Path::from(name_1), last_modified: Utc::now(), - size: total_size_2, + size: total_size_1, e_tag: None, version: None, }, @@ -2504,13 +2172,25 @@ mod tests { statistics: None, extensions: None, metadata_size_hint: None, - }), - ) - .with_parquet_file_reader_factory(reader_factory) - .with_metadata_size_hint(456) - .build(); + } + .with_metadata_size_hint(123), + ) + .with_file(PartitionedFile { + object_meta: ObjectMeta { + location: Path::from(name_2), + last_modified: Utc::now(), + size: total_size_2, + e_tag: None, + version: None, + }, + partition_values: vec![], + range: None, + statistics: None, + extensions: None, + metadata_size_hint: None, + }) + .new_exec(); - let exec = Arc::new(exec); let res = collect(exec, ctx.task_ctx()).await.unwrap(); assert_eq!(res.len(), 2); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs index a1f8f0172ce4..02ad9dd55100 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs @@ -295,7 +295,7 @@ fn create_initial_plan( // check row group count matches the plan return Ok(access_plan.clone()); } else { - debug!("ParquetExec Ignoring unknown extension specified for {file_name}"); + debug!("DataSourceExec Ignoring unknown extension specified for {file_name}"); } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index f6428a693fb1..bcd2c0af6f6f 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -386,8 +386,8 @@ impl TreeNodeRewriter for PushdownChecker<'_> { type ProjectionAndExpr = (BTreeSet, Arc); -// Checks if a given expression can be pushed down into `ParquetExec` as opposed to being evaluated -// post-parquet-scan in a `FilterExec`. If it can be pushed down, this returns returns all the +// Checks if a given expression can be pushed down into `DataSourceExec` as opposed to being evaluated +// post-parquet-scan in a `FilterExec`. If it can be pushed down, this returns all the // columns in the given expression so that they can be used in the parquet scanning, along with the // expression rewritten as defined in [`PushdownChecker::f_up`] fn pushdown_columns( diff --git a/datafusion/core/src/datasource/physical_plan/parquet/source.rs b/datafusion/core/src/datasource/physical_plan/parquet/source.rs new file mode 100644 index 000000000000..c00fe91b859f --- /dev/null +++ b/datafusion/core/src/datasource/physical_plan/parquet/source.rs @@ -0,0 +1,589 @@ +// 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. + +//! ParquetSource implementation for reading parquet files +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + +use crate::datasource::data_source::FileSource; +use crate::datasource::physical_plan::parquet::opener::ParquetOpener; +use crate::datasource::physical_plan::parquet::page_filter::PagePruningAccessPlanFilter; +use crate::datasource::physical_plan::parquet::DefaultParquetFileReaderFactory; +use crate::datasource::physical_plan::{ + FileOpener, FileScanConfig, ParquetFileReaderFactory, +}; +use crate::datasource::schema_adapter::{ + DefaultSchemaAdapterFactory, SchemaAdapterFactory, +}; + +use arrow_schema::{Schema, SchemaRef}; +use datafusion_common::config::TableParquetOptions; +use datafusion_common::Statistics; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_optimizer::pruning::PruningPredicate; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; +use datafusion_physical_plan::DisplayFormatType; + +use itertools::Itertools; +use log::debug; +use object_store::ObjectStore; + +/// Execution plan for reading one or more Parquet files. +/// +/// ```text +/// ▲ +/// │ +/// │ Produce a stream of +/// │ RecordBatches +/// │ +/// ┌───────────────────────┐ +/// │ │ +/// │ DataSourceExec │ +/// │ │ +/// └───────────────────────┘ +/// ▲ +/// │ Asynchronously read from one +/// │ or more parquet files via +/// │ ObjectStore interface +/// │ +/// │ +/// .───────────────────. +/// │ ) +/// │`───────────────────'│ +/// │ ObjectStore │ +/// │.───────────────────.│ +/// │ ) +/// `───────────────────' +/// +/// ``` +/// +/// # Example: Create a `DataSourceExec` +/// ``` +/// # use std::sync::Arc; +/// # use arrow::datatypes::Schema; +/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; +/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion_execution::object_store::ObjectStoreUrl; +/// # use datafusion_physical_expr::expressions::lit; +/// # use datafusion_physical_plan::source::DataSourceExec; +/// # use datafusion_common::config::TableParquetOptions; +/// +/// # let file_schema = Arc::new(Schema::empty()); +/// # let object_store_url = ObjectStoreUrl::local_filesystem(); +/// # let predicate = lit(true); +/// let source = Arc::new( +/// ParquetSource::default() +/// .with_predicate(Arc::clone(&file_schema), predicate) +/// ); +/// // Create a DataSourceExec for reading `file1.parquet` with a file size of 100MB +/// let file_scan_config = FileScanConfig::new(object_store_url, file_schema, source) +/// .with_file(PartitionedFile::new("file1.parquet", 100*1024*1024)); +/// let exec = file_scan_config.new_exec(); +/// ``` +/// +/// # Features +/// +/// Supports the following optimizations: +/// +/// * Concurrent reads: reads from one or more files in parallel as multiple +/// partitions, including concurrently reading multiple row groups from a single +/// file. +/// +/// * Predicate push down: skips row groups, pages, rows based on metadata +/// and late materialization. See "Predicate Pushdown" below. +/// +/// * Projection pushdown: reads and decodes only the columns required. +/// +/// * Limit pushdown: stop execution early after some number of rows are read. +/// +/// * Custom readers: customize reading parquet files, e.g. to cache metadata, +/// coalesce I/O operations, etc. See [`ParquetFileReaderFactory`] for more +/// details. +/// +/// * Schema evolution: read parquet files with different schemas into a unified +/// table schema. See [`SchemaAdapterFactory`] for more details. +/// +/// * metadata_size_hint: controls the number of bytes read from the end of the +/// file in the initial I/O when the default [`ParquetFileReaderFactory`]. If a +/// custom reader is used, it supplies the metadata directly and this parameter +/// is ignored. [`ParquetSource::with_metadata_size_hint`] for more details. +/// +/// * User provided `ParquetAccessPlan`s to skip row groups and/or pages +/// based on external information. See "Implementing External Indexes" below +/// +/// # Predicate Pushdown +/// +/// `DataSourceExec` uses the provided [`PhysicalExpr`] predicate as a filter to +/// skip reading unnecessary data and improve query performance using several techniques: +/// +/// * Row group pruning: skips entire row groups based on min/max statistics +/// found in [`ParquetMetaData`] and any Bloom filters that are present. +/// +/// * Page pruning: skips individual pages within a ColumnChunk using the +/// [Parquet PageIndex], if present. +/// +/// * Row filtering: skips rows within a page using a form of late +/// materialization. When possible, predicates are applied by the parquet +/// decoder *during* decode (see [`ArrowPredicate`] and [`RowFilter`] for more +/// details). This is only enabled if `ParquetScanOptions::pushdown_filters` is set to true. +/// +/// Note: If the predicate can not be used to accelerate the scan, it is ignored +/// (no error is raised on predicate evaluation errors). +/// +/// [`ArrowPredicate`]: parquet::arrow::arrow_reader::ArrowPredicate +/// [`RowFilter`]: parquet::arrow::arrow_reader::RowFilter +/// [Parquet PageIndex]: https://github.com/apache/parquet-format/blob/master/PageIndex.md +/// +/// # Example: rewriting `DataSourceExec` +/// +/// You can modify a `DataSourceExec` using [`ParquetSource`], for example +/// to change files or add a predicate. +/// +/// ```no_run +/// # use std::sync::Arc; +/// # use arrow::datatypes::Schema; +/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion_physical_plan::source::DataSourceExec; +/// +/// # fn parquet_exec() -> DataSourceExec { unimplemented!() } +/// // Split a single DataSourceExec into multiple DataSourceExecs, one for each file +/// let exec = parquet_exec(); +/// let source = exec.source(); +/// let base_config = source.as_any().downcast_ref::().unwrap(); +/// let existing_file_groups = &base_config.file_groups; +/// let new_execs = existing_file_groups +/// .iter() +/// .map(|file_group| { +/// // create a new exec by copying the existing exec's source config +/// let new_config = base_config +/// .clone() +/// .with_file_groups(vec![file_group.clone()]); +/// +/// new_config.new_exec() +/// }) +/// .collect::>(); +/// ``` +/// +/// # Implementing External Indexes +/// +/// It is possible to restrict the row groups and selections within those row +/// groups that the DataSourceExec will consider by providing an initial +/// `ParquetAccessPlan` as `extensions` on `PartitionedFile`. This can be +/// used to implement external indexes on top of parquet files and select only +/// portions of the files. +/// +/// The `DataSourceExec` will try and reduce any provided `ParquetAccessPlan` +/// further based on the contents of `ParquetMetadata` and other settings. +/// +/// ## Example of providing a ParquetAccessPlan +/// +/// ``` +/// # use std::sync::Arc; +/// # use arrow_schema::{Schema, SchemaRef}; +/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; +/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; +/// # use datafusion_execution::object_store::ObjectStoreUrl; +/// # use datafusion_physical_plan::source::DataSourceExec; +/// +/// # fn schema() -> SchemaRef { +/// # Arc::new(Schema::empty()) +/// # } +/// // create an access plan to scan row group 0, 1 and 3 and skip row groups 2 and 4 +/// let mut access_plan = ParquetAccessPlan::new_all(5); +/// access_plan.skip(2); +/// access_plan.skip(4); +/// // provide the plan as extension to the FileScanConfig +/// let partitioned_file = PartitionedFile::new("my_file.parquet", 1234) +/// .with_extensions(Arc::new(access_plan)); +/// // create a FileScanConfig to scan this file +/// let file_scan_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema(), Arc::new(ParquetSource::default())) +/// .with_file(partitioned_file); +/// // this parquet DataSourceExec will not even try to read row groups 2 and 4. Additional +/// // pruning based on predicates may also happen +/// let exec = file_scan_config.new_exec(); +/// ``` +/// +/// For a complete example, see the [`advanced_parquet_index` example]). +/// +/// [`parquet_index_advanced` example]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs +/// +/// # Execution Overview +/// +/// * Step 1: `DataSourceExec::execute` is called, returning a `FileStream` +/// configured to open parquet files with a `ParquetOpener`. +/// +/// * Step 2: When the stream is polled, the `ParquetOpener` is called to open +/// the file. +/// +/// * Step 3: The `ParquetOpener` gets the [`ParquetMetaData`] (file metadata) +/// via [`ParquetFileReaderFactory`], creating a `ParquetAccessPlan` by +/// applying predicates to metadata. The plan and projections are used to +/// determine what pages must be read. +/// +/// * Step 4: The stream begins reading data, fetching the required parquet +/// pages incrementally decoding them, and applying any row filters (see +/// [`Self::with_pushdown_filters`]). +/// +/// * Step 5: As each [`RecordBatch`] is read, it may be adapted by a +/// [`SchemaAdapter`] to match the table schema. By default missing columns are +/// filled with nulls, but this can be customized via [`SchemaAdapterFactory`]. +/// +/// [`RecordBatch`]: arrow::record_batch::RecordBatch +/// [`SchemaAdapter`]: crate::datasource::schema_adapter::SchemaAdapter +/// [`ParquetMetadata`]: parquet::file::metadata::ParquetMetaData +#[derive(Clone, Default, Debug)] +pub struct ParquetSource { + /// Options for reading Parquet files + pub(crate) table_parquet_options: TableParquetOptions, + /// Optional metrics + pub(crate) metrics: ExecutionPlanMetricsSet, + /// Optional predicate for row filtering during parquet scan + pub(crate) predicate: Option>, + /// Optional predicate for pruning row groups (derived from `predicate`) + pub(crate) pruning_predicate: Option>, + /// Optional predicate for pruning pages (derived from `predicate`) + pub(crate) page_pruning_predicate: Option>, + /// Optional user defined parquet file reader factory + pub(crate) parquet_file_reader_factory: Option>, + /// Optional user defined schema adapter + pub(crate) schema_adapter_factory: Option>, + /// Batch size configuration + pub(crate) batch_size: Option, + /// Optional hint for the size of the parquet metadata + pub(crate) metadata_size_hint: Option, + pub(crate) projected_statistics: Option, +} + +impl ParquetSource { + /// Create a new ParquetSource to read the data specified in the file scan + /// configuration with the provided `TableParquetOptions`. + /// if default values are going to be used, use `ParguetConfig::default()` instead + pub fn new(table_parquet_options: TableParquetOptions) -> Self { + Self { + table_parquet_options, + ..Self::default() + } + } + + /// Set the metadata size hint + /// + /// This value determines how many bytes at the end of the file the default + /// [`ParquetFileReaderFactory`] will request in the initial IO. If this is + /// too small, the ParquetSource will need to make additional IO requests to + /// read the footer. + pub fn with_metadata_size_hint(mut self, metadata_size_hint: usize) -> Self { + self.metadata_size_hint = Some(metadata_size_hint); + self + } + + fn with_metrics(&self, metrics: ExecutionPlanMetricsSet) -> Self { + let mut conf = self.clone(); + conf.metrics = metrics; + conf + } + + /// Set predicate information, also sets pruning_predicate and page_pruning_predicate attributes + pub fn with_predicate( + &self, + file_schema: Arc, + predicate: Arc, + ) -> Self { + let mut conf = self.clone(); + + let metrics = ExecutionPlanMetricsSet::new(); + let predicate_creation_errors = + MetricBuilder::new(&metrics).global_counter("num_predicate_creation_errors"); + + conf.with_metrics(metrics); + conf.predicate = Some(Arc::clone(&predicate)); + + match PruningPredicate::try_new(Arc::clone(&predicate), Arc::clone(&file_schema)) + { + Ok(pruning_predicate) => { + if !pruning_predicate.always_true() { + conf.pruning_predicate = Some(Arc::new(pruning_predicate)); + } + } + Err(e) => { + debug!("Could not create pruning predicate for: {e}"); + predicate_creation_errors.add(1); + } + }; + + let page_pruning_predicate = Arc::new(PagePruningAccessPlanFilter::new( + &predicate, + Arc::clone(&file_schema), + )); + conf.page_pruning_predicate = Some(page_pruning_predicate); + + conf + } + + /// Options passed to the parquet reader for this scan + pub fn table_parquet_options(&self) -> &TableParquetOptions { + &self.table_parquet_options + } + + /// Optional predicate. + pub fn predicate(&self) -> Option<&Arc> { + self.predicate.as_ref() + } + + /// Optional reference to this parquet scan's pruning predicate + pub fn pruning_predicate(&self) -> Option<&Arc> { + self.pruning_predicate.as_ref() + } + + /// Optional reference to this parquet scan's page pruning predicate + pub fn page_pruning_predicate(&self) -> Option<&Arc> { + self.page_pruning_predicate.as_ref() + } + + /// return the optional file reader factory + pub fn parquet_file_reader_factory( + &self, + ) -> Option<&Arc> { + self.parquet_file_reader_factory.as_ref() + } + + /// Optional user defined parquet file reader factory. + /// + pub fn with_parquet_file_reader_factory( + mut self, + parquet_file_reader_factory: Arc, + ) -> Self { + self.parquet_file_reader_factory = Some(parquet_file_reader_factory); + self + } + + /// return the optional schema adapter factory + pub fn schema_adapter_factory(&self) -> Option<&Arc> { + self.schema_adapter_factory.as_ref() + } + + /// Set optional schema adapter factory. + /// + /// [`SchemaAdapterFactory`] allows user to specify how fields from the + /// parquet file get mapped to that of the table schema. The default schema + /// adapter uses arrow's cast library to map the parquet fields to the table + /// schema. + pub fn with_schema_adapter_factory( + mut self, + schema_adapter_factory: Arc, + ) -> Self { + self.schema_adapter_factory = Some(schema_adapter_factory); + self + } + + /// If true, the predicate will be used during the parquet scan. + /// Defaults to false + /// + /// [`Expr`]: datafusion_expr::Expr + pub fn with_pushdown_filters(mut self, pushdown_filters: bool) -> Self { + self.table_parquet_options.global.pushdown_filters = pushdown_filters; + self + } + + /// Return the value described in [`Self::with_pushdown_filters`] + pub(crate) fn pushdown_filters(&self) -> bool { + self.table_parquet_options.global.pushdown_filters + } + + /// If true, the `RowFilter` made by `pushdown_filters` may try to + /// minimize the cost of filter evaluation by reordering the + /// predicate [`Expr`]s. If false, the predicates are applied in + /// the same order as specified in the query. Defaults to false. + /// + /// [`Expr`]: datafusion_expr::Expr + pub fn with_reorder_filters(mut self, reorder_filters: bool) -> Self { + self.table_parquet_options.global.reorder_filters = reorder_filters; + self + } + + /// Return the value described in [`Self::with_reorder_filters`] + fn reorder_filters(&self) -> bool { + self.table_parquet_options.global.reorder_filters + } + + /// If enabled, the reader will read the page index + /// This is used to optimize filter pushdown + /// via `RowSelector` and `RowFilter` by + /// eliminating unnecessary IO and decoding + pub fn with_enable_page_index(mut self, enable_page_index: bool) -> Self { + self.table_parquet_options.global.enable_page_index = enable_page_index; + self + } + + /// Return the value described in [`Self::with_enable_page_index`] + fn enable_page_index(&self) -> bool { + self.table_parquet_options.global.enable_page_index + } + + /// If enabled, the reader will read by the bloom filter + pub fn with_bloom_filter_on_read(mut self, bloom_filter_on_read: bool) -> Self { + self.table_parquet_options.global.bloom_filter_on_read = bloom_filter_on_read; + self + } + + /// If enabled, the writer will write by the bloom filter + pub fn with_bloom_filter_on_write( + mut self, + enable_bloom_filter_on_write: bool, + ) -> Self { + self.table_parquet_options.global.bloom_filter_on_write = + enable_bloom_filter_on_write; + self + } + + /// Return the value described in [`Self::with_bloom_filter_on_read`] + fn bloom_filter_on_read(&self) -> bool { + self.table_parquet_options.global.bloom_filter_on_read + } +} + +impl FileSource for ParquetSource { + fn create_file_opener( + &self, + object_store: datafusion_common::Result>, + base_config: &FileScanConfig, + partition: usize, + ) -> datafusion_common::Result> { + let projection = base_config + .file_column_projection_indices() + .unwrap_or_else(|| (0..base_config.file_schema.fields().len()).collect()); + let schema_adapter_factory = self + .schema_adapter_factory + .clone() + .unwrap_or_else(|| Arc::new(DefaultSchemaAdapterFactory)); + + let parquet_file_reader_factory = self + .parquet_file_reader_factory + .as_ref() + .map(|f| Ok(Arc::clone(f))) + .unwrap_or_else(|| { + object_store.map(|store| { + Arc::new(DefaultParquetFileReaderFactory::new(store)) as _ + }) + })?; + + Ok(Arc::new(ParquetOpener { + partition_index: partition, + projection: Arc::from(projection), + batch_size: self + .batch_size + .expect("Batch size must set before creating ParquetOpener"), + limit: base_config.limit, + predicate: self.predicate.clone(), + pruning_predicate: self.pruning_predicate.clone(), + page_pruning_predicate: self.page_pruning_predicate.clone(), + table_schema: Arc::clone(&base_config.file_schema), + metadata_size_hint: self.metadata_size_hint, + metrics: self.metrics().clone(), + parquet_file_reader_factory, + pushdown_filters: self.pushdown_filters(), + reorder_filters: self.reorder_filters(), + enable_page_index: self.enable_page_index(), + enable_bloom_filter: self.bloom_filter_on_read(), + schema_adapter_factory, + })) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn with_statistics(&self, statistics: Statistics) -> Arc { + let mut conf = self.clone(); + conf.projected_statistics = Some(statistics); + Arc::new(conf) + } + + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + Arc::new(Self { ..self.clone() }) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + &self.metrics + } + + fn statistics(&self) -> datafusion_common::Result { + let statistics = &self.projected_statistics; + let statistics = statistics + .clone() + .expect("projected_statistics must be set"); + // When filters are pushed down, we have no way of knowing the exact statistics. + // Note that pruning predicate is also a kind of filter pushdown. + // (bloom filters use `pruning_predicate` too) + if self.pruning_predicate().is_some() + || self.page_pruning_predicate().is_some() + || (self.predicate().is_some() && self.pushdown_filters()) + { + Ok(statistics.to_inexact()) + } else { + Ok(statistics) + } + } + + fn file_type(&self) -> &str { + "parquet" + } + + fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let predicate_string = self + .predicate() + .map(|p| format!(", predicate={p}")) + .unwrap_or_default(); + + let pruning_predicate_string = self + .pruning_predicate() + .map(|pre| { + let mut guarantees = pre + .literal_guarantees() + .iter() + .map(|item| format!("{}", item)) + .collect_vec(); + guarantees.sort(); + format!( + ", pruning_predicate={}, required_guarantees=[{}]", + pre.predicate_expr(), + guarantees.join(", ") + ) + }) + .unwrap_or_default(); + + write!(f, "{}{}", predicate_string, pruning_predicate_string) + } + } + } +} diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index b27cf9c5f833..7e41e450ce23 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -31,7 +31,7 @@ use std::sync::Arc; /// Factory for creating [`SchemaAdapter`] /// /// This interface provides a way to implement custom schema adaptation logic -/// for ParquetExec (for example, to fill missing columns with default value +/// for DataSourceExec (for example, to fill missing columns with default value /// other than null). /// /// Most users should use [`DefaultSchemaAdapterFactory`]. See that struct for @@ -229,7 +229,7 @@ impl SchemaAdapterFactory for DefaultSchemaAdapterFactory { #[derive(Clone, Debug)] pub(crate) struct DefaultSchemaAdapter { /// The schema for the table, projected to include only the fields being output (projected) by the - /// associated ParquetExec + /// associated ParquetSource projected_table_schema: SchemaRef, /// The entire table schema for the table we're using this to adapt. /// @@ -315,7 +315,7 @@ impl SchemaAdapter for DefaultSchemaAdapter { /// can be used for Parquet predicate pushdown, meaning that it may contain /// fields which are not in the projected schema (as the fields that parquet /// pushdown filters operate can be completely distinct from the fields that are -/// projected (output) out of the ParquetExec). `map_partial_batch` thus uses +/// projected (output) out of the ParquetSource). `map_partial_batch` thus uses /// `table_schema` to create the resulting RecordBatch (as it could be operating /// on any fields in the schema). /// @@ -441,15 +441,15 @@ mod tests { use object_store::path::Path; use object_store::ObjectMeta; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; - use crate::physical_plan::collect; - use crate::prelude::SessionContext; - use crate::datasource::listing::PartitionedFile; + use crate::datasource::object_store::ObjectStoreUrl; + use crate::datasource::physical_plan::{FileScanConfig, ParquetSource}; use crate::datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; + use crate::physical_plan::collect; + use crate::prelude::SessionContext; + use datafusion_common::record_batch; #[cfg(feature = "parquet")] use parquet::arrow::ArrowWriter; @@ -500,18 +500,19 @@ mod tests { let f2 = Field::new("extra_column", DataType::Utf8, true); let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); + let source = Arc::new( + ParquetSource::default() + .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})), + ); + let base_conf = + FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema, source) + .with_file(partitioned_file); - // prepare the scan - let parquet_exec = ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema) - .with_file(partitioned_file), - ) - .build() - .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})); + let parquet_exec = base_conf.new_exec(); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let read = collect(Arc::new(parquet_exec), task_ctx).await.unwrap(); + let read = collect(parquet_exec, task_ctx).await.unwrap(); let expected = [ "+----+--------------+", diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index 33a3f4da6843..91e9b6789fda 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -504,11 +504,12 @@ mod tests { .select_columns(&["bool_col", "int_col"])?; let plan = df.explain(false, false)?.collect().await?; - // Limit is included in ParquetExec + // Limit is included in DataSourceExec let formatted = arrow::util::pretty::pretty_format_batches(&plan) .unwrap() .to_string(); - assert!(formatted.contains("ParquetExec: ")); + assert!(formatted.contains("DataSourceExec: ")); + assert!(formatted.contains("file_type=parquet")); assert!(formatted.contains("projection=[bool_col, int_col], limit=10")); Ok(()) } diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index ca0aa92ff1ed..9d42580178a4 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -249,11 +249,11 @@ //! AnalyzerRules and PhysicalPlanner PhysicalOptimizerRules //! OptimizerRules creates ExecutionPlan improve performance //! rewrite plan -//! ┌─────────────┐ ┌─────────────┐ ┌───────────────┐ ┌───────────────┐ -//! │Project │ │Project(x, y)│ │ProjectExec │ │ProjectExec │ -//! │ TableScan │──...──▶│ TableScan │─────▶│ ... │──...──▶│ ... │ -//! │ ... │ │ ... │ │ ParquetExec│ │ ParquetExec│ -//! └─────────────┘ └─────────────┘ └───────────────┘ └───────────────┘ +//! ┌─────────────┐ ┌─────────────┐ ┌─────────────────┐ ┌─────────────────┐ +//! │Project │ │Project(x, y)│ │ProjectExec │ │ProjectExec │ +//! │ TableScan │──...──▶│ TableScan │─────▶│ ... │──...──▶│ ... │ +//! │ ... │ │ ... │ │ DataSourceExec│ │ DataSourceExec│ +//! └─────────────┘ └─────────────┘ └─────────────────┘ └─────────────────┘ //! //! LogicalPlan LogicalPlan ExecutionPlan ExecutionPlan //! ``` @@ -284,11 +284,11 @@ //! such as schema │ ExecutionPlan //! │ //! ▼ -//! ┌─────────────────────────┐ ┌──────────────┐ -//! │ │ │ │ -//! │impl TableProvider │────────▶│ParquetExec │ -//! │ │ │ │ -//! └─────────────────────────┘ └──────────────┘ +//! ┌─────────────────────────┐ ┌───────────────┐ +//! │ │ │ │ +//! │impl TableProvider │────────▶│DataSourceExec │ +//! │ │ │ │ +//! └─────────────────────────┘ └───────────────┘ //! TableProvider //! (built in or user provided) ExecutionPlan //! ``` @@ -358,20 +358,20 @@ //! ExecutionPlan::execute Calling next() on the //! produces a stream stream produces the data //! -//! ┌───────────────┐ ┌─────────────────────────┐ ┌────────────┐ -//! │ProjectExec │ │impl │ ┌───▶│RecordBatch │ -//! │ ... │─────▶│SendableRecordBatchStream│────┤ └────────────┘ -//! │ ParquetExec│ │ │ │ ┌────────────┐ -//! └───────────────┘ └─────────────────────────┘ ├───▶│RecordBatch │ -//! ▲ │ └────────────┘ -//! ExecutionPlan │ │ ... -//! │ │ -//! │ │ ┌────────────┐ -//! PhysicalOptimizerRules ├───▶│RecordBatch │ -//! request information │ └────────────┘ -//! such as partitioning │ ┌ ─ ─ ─ ─ ─ ─ -//! └───▶ None │ -//! └ ─ ─ ─ ─ ─ ─ +//! ┌────────────────┐ ┌─────────────────────────┐ ┌────────────┐ +//! │ProjectExec │ │impl │ ┌───▶│RecordBatch │ +//! │ ... │─────▶│SendableRecordBatchStream│────┤ └────────────┘ +//! │ DataSourceExec│ │ │ │ ┌────────────┐ +//! └────────────────┘ └─────────────────────────┘ ├───▶│RecordBatch │ +//! ▲ │ └────────────┘ +//! ExecutionPlan │ │ ... +//! │ │ +//! │ │ ┌────────────┐ +//! PhysicalOptimizerRules ├───▶│RecordBatch │ +//! request information │ └────────────┘ +//! such as partitioning │ ┌ ─ ─ ─ ─ ─ ─ +//! └───▶ None │ +//! └ ─ ─ ─ ─ ─ ─ //! ``` //! //! [`ExecutionPlan`]s process data using the [Apache Arrow] memory @@ -442,14 +442,14 @@ //! ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ //! │ Step 1: Consumer //! ▼ ▼ │ calls next() -//! ┏━━━━━━━━━━━━━━┓ ┏━━━━━┻━━━━━━━━━━━━━┓ ┏━━━━━━━━━━━━━━━━━━━━━━━━┓ -//! ┃ ┃ ┃ ┃ ┃ ◀ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ -//! ┃ DataSource ┃ ┃ ┃ ┃ ┃ -//! ┃ (e.g. ┃ ┃ FilterExec ┃ ┃ ProjectionExec ┃ -//! ┃ ParquetExec) ┃ ┃id IN (10, 20, 30) ┃ ┃date_bin('month', time) ┃ -//! ┃ ┃ ┃ ┃ ┃ ┣ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ▶ -//! ┃ ┃ ┃ ┃ ┃ ┃ -//! ┗━━━━━━━━━━━━━━┛ ┗━━━━━━━━━━━┳━━━━━━━┛ ┗━━━━━━━━━━━━━━━━━━━━━━━━┛ +//! ┏━━━━━━━━━━━━━━━━┓ ┏━━━━━┻━━━━━━━━━━━━━┓ ┏━━━━━━━━━━━━━━━━━━━━━━━━┓ +//! ┃ ┃ ┃ ┃ ┃ ◀ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +//! ┃ DataSource ┃ ┃ ┃ ┃ ┃ +//! ┃ (e.g. ┃ ┃ FilterExec ┃ ┃ ProjectionExec ┃ +//! ┃ ParquetSource) ┃ ┃id IN (10, 20, 30) ┃ ┃date_bin('month', time) ┃ +//! ┃ ┃ ┃ ┃ ┃ ┣ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ▶ +//! ┃ ┃ ┃ ┃ ┃ ┃ +//! ┗━━━━━━━━━━━━━━━━┛ ┗━━━━━━━━━━━┳━━━━━━━┛ ┗━━━━━━━━━━━━━━━━━━━━━━━━┛ //! │ ▲ ▲ Step 6: ProjectionExec //! │ │ │ computes date_trunc into a //! └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ new RecordBatch returned diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 5f98ba4efcf0..cee3acc08dae 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -47,7 +47,6 @@ use crate::physical_plan::joins::{ CrossJoinExec, HashJoinExec, NestedLoopJoinExec, PartitionMode, SortMergeJoinExec, }; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::recursive_query::RecursiveQueryExec; use crate::physical_plan::repartition::RepartitionExec; @@ -83,13 +82,14 @@ use datafusion_expr::{ use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::execution_plan::InvariantLevel; +use datafusion_physical_plan::memory::MemorySourceConfig; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::unnest::ListUnnest; use datafusion_sql::utils::window_expr_common_partition_keys; use async_trait::async_trait; -use datafusion_physical_optimizer::PhysicalOptimizerRule; use futures::{StreamExt, TryStreamExt}; use itertools::{multiunzip, Itertools}; use log::{debug, trace}; @@ -467,9 +467,8 @@ impl DefaultPhysicalPlanner { .collect::>>>() }) .collect::>>()?; - let value_exec = - MemoryExec::try_new_as_values(SchemaRef::new(exec_schema), exprs)?; - Arc::new(value_exec) + MemorySourceConfig::try_new_as_values(SchemaRef::new(exec_schema), exprs)? + as _ } LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, @@ -1946,8 +1945,8 @@ impl DefaultPhysicalPlanner { let schema = record_batch.schema(); let partitions = vec![vec![record_batch]]; let projection = None; - let mem_exec = MemoryExec::try_new(&partitions, schema, projection)?; - Ok(Arc::new(mem_exec)) + let mem_exec = MemorySourceConfig::try_new_exec(&partitions, schema, projection)?; + Ok(mem_exec) } fn create_project_physical_exec( diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 05e63a3c4fd4..f2fef06c8f30 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -25,12 +25,13 @@ use std::io::{BufReader, BufWriter}; use std::path::Path; use std::sync::Arc; +use crate::datasource::data_source::FileSource; use crate::datasource::file_format::csv::CsvFormat; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::FileFormat; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; -use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; +use crate::datasource::physical_plan::{CsvSource, FileScanConfig}; use crate::datasource::{MemTable, TableProvider}; use crate::error::Result; use crate::logical_expr::LogicalPlan; @@ -41,6 +42,7 @@ use arrow::array::{self, Array, ArrayRef, Decimal128Builder, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::DataFusionError; +use datafusion_physical_plan::source::DataSourceExec; #[cfg(feature = "compression")] use bzip2::write::BzEncoder; @@ -72,8 +74,11 @@ pub fn create_table_dual() -> Arc { Arc::new(provider) } -/// Returns a [`CsvExec`] that scans "aggregate_test_100.csv" with `partitions` partitions -pub fn scan_partitioned_csv(partitions: usize, work_dir: &Path) -> Result> { +/// Returns a [`DataSourceExec`] that scans "aggregate_test_100.csv" with `partitions` partitions +pub fn scan_partitioned_csv( + partitions: usize, + work_dir: &Path, +) -> Result> { let schema = aggr_test_schema(); let filename = "aggregate_test_100.csv"; let path = format!("{}/csv", arrow_test_data()); @@ -85,18 +90,10 @@ pub fn scan_partitioned_csv(partitions: usize, work_dir: &Path) -> Result>, + source: Arc, ) -> FileScanConfig { - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema) + FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema, source) .with_file_groups(file_groups) } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 685ed14777b4..67e0e1726917 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -26,7 +26,7 @@ use crate::common::ToDFSchema; use crate::config::ConfigOptions; use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; use crate::datasource::object_store::ObjectStoreUrl; -use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use crate::datasource::physical_plan::{FileScanConfig, ParquetSource}; use crate::error::Result; use crate::logical_expr::execution_props::ExecutionProps; use crate::logical_expr::simplify::SimplifyContext; @@ -37,7 +37,7 @@ use crate::physical_plan::metrics::MetricsSet; use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig, SessionContext}; -use crate::datasource::physical_plan::parquet::ParquetExecBuilder; +use datafusion_physical_plan::source::DataSourceExec; use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; @@ -137,68 +137,82 @@ impl TestParquetFile { } impl TestParquetFile { - /// Return a `ParquetExec` with the specified options. + /// Return a `DataSourceExec` with the specified options. /// - /// If `maybe_filter` is non-None, the ParquetExec will be filtered using + /// If `maybe_filter` is non-None, the DataSourceExec will be filtered using /// the given expression, and this method will return the same plan that DataFusion /// will make with a pushed down predicate followed by a filter: /// /// ```text /// (FilterExec) - /// (ParquetExec) + /// (DataSourceExec) /// ``` /// - /// Otherwise if `maybe_filter` is None, return just a `ParquetExec` + /// Otherwise if `maybe_filter` is None, return just a `DataSourceExec` pub async fn create_scan( &self, ctx: &SessionContext, maybe_filter: Option, ) -> Result> { - let scan_config = - FileScanConfig::new(self.object_store_url.clone(), Arc::clone(&self.schema)) - .with_file(PartitionedFile { - object_meta: self.object_meta.clone(), - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - }); + let parquet_options = ctx.copied_table_options().parquet; + let source = Arc::new(ParquetSource::new(parquet_options.clone())); + let mut scan_config = FileScanConfig::new( + self.object_store_url.clone(), + Arc::clone(&self.schema), + source, + ) + .with_file(PartitionedFile { + object_meta: self.object_meta.clone(), + partition_values: vec![], + range: None, + statistics: None, + extensions: None, + metadata_size_hint: None, + }); let df_schema = Arc::clone(&self.schema).to_dfschema_ref()?; // run coercion on the filters to coerce types etc. let props = ExecutionProps::new(); let context = SimplifyContext::new(&props).with_schema(Arc::clone(&df_schema)); - let parquet_options = ctx.copied_table_options().parquet; if let Some(filter) = maybe_filter { let simplifier = ExprSimplifier::new(context); let filter = simplifier.coerce(filter, &df_schema).unwrap(); let physical_filter_expr = create_physical_expr(&filter, &df_schema, &ExecutionProps::default())?; - let parquet_exec = - ParquetExecBuilder::new_with_options(scan_config, parquet_options) - .with_predicate(Arc::clone(&physical_filter_expr)) - .build_arc(); + let source = Arc::new(ParquetSource::new(parquet_options).with_predicate( + Arc::clone(&scan_config.file_schema), + Arc::clone(&physical_filter_expr), + )); + scan_config = scan_config.with_source(source); + let parquet_exec = scan_config.new_exec(); let exec = Arc::new(FilterExec::try_new(physical_filter_expr, parquet_exec)?); Ok(exec) } else { - Ok( - ParquetExecBuilder::new_with_options(scan_config, parquet_options) - .build_arc(), - ) + Ok(scan_config.new_exec()) } } /// Retrieve metrics from the parquet exec returned from `create_scan` /// - /// Recursively searches for ParquetExec and returns the metrics + /// Recursively searches for DataSourceExec and returns the metrics /// on the first one it finds pub fn parquet_metrics(plan: &Arc) -> Option { - if let Some(parquet) = plan.as_any().downcast_ref::() { - return parquet.metrics(); + if let Some(maybe_file) = plan.as_any().downcast_ref::() { + let source = maybe_file.source(); + if let Some(maybe_parquet) = source.as_any().downcast_ref::() + { + if maybe_parquet + .file_source() + .as_any() + .downcast_ref::() + .is_some() + { + return maybe_file.metrics(); + } + } } for child in plan.children() { diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index d6ba4d5337c6..acd5ee6d5ef0 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -549,7 +549,7 @@ async fn test_aggregate_with_pk() -> Result<()> { &df, vec![ "AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " DataSourceExec: partitions=1, partition_sizes=[1]", ], ) .await; @@ -593,7 +593,7 @@ async fn test_aggregate_with_pk2() -> Result<()> { "CoalesceBatchesExec: target_batch_size=8192", " FilterExec: id@0 = 1 AND name@1 = a", " AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " DataSourceExec: partitions=1, partition_sizes=[1]", ], ) .await; @@ -642,7 +642,7 @@ async fn test_aggregate_with_pk3() -> Result<()> { "CoalesceBatchesExec: target_batch_size=8192", " FilterExec: id@0 = 1", " AggregateExec: mode=Single, gby=[id@0 as id, name@1 as name], aggr=[]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " DataSourceExec: partitions=1, partition_sizes=[1]", ], ) .await; @@ -693,7 +693,7 @@ async fn test_aggregate_with_pk4() -> Result<()> { "CoalesceBatchesExec: target_batch_size=8192", " FilterExec: id@0 = 1", " AggregateExec: mode=Single, gby=[id@0 as id], aggr=[]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " DataSourceExec: partitions=1, partition_sizes=[1]", ], ) .await; diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index bcd88bae739a..a58855438475 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -15,8 +15,13 @@ // specific language governing permissions and limitations // under the License. +use std::str; use std::sync::Arc; +use crate::fuzz_cases::aggregation_fuzzer::{ + AggregationFuzzerBuilder, ColumnDescr, DatasetGeneratorConfig, QueryBuilder, +}; + use arrow::array::{Array, ArrayRef, AsArray, Int64Array}; use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::DataType; @@ -33,24 +38,21 @@ use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; -use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::{collect, displayable, ExecutionPlan}; use datafusion::prelude::{DataFrame, SessionConfig, SessionContext}; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; +use datafusion_common::HashMap; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::InputOrderMode; use test_utils::{add_empty_batches, StringBatchGenerator}; -use crate::fuzz_cases::aggregation_fuzzer::{ - AggregationFuzzerBuilder, ColumnDescr, DatasetGeneratorConfig, QueryBuilder, -}; -use datafusion_common::HashMap; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use rand::rngs::StdRng; use rand::{thread_rng, Rng, SeedableRng}; -use std::str; use tokio::task::JoinSet; // ======================================================================== @@ -321,16 +323,20 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str } let concat_input_record = concat_batches(&schema, &input1).unwrap(); - let usual_source = Arc::new( - MemoryExec::try_new(&[vec![concat_input_record]], schema.clone(), None).unwrap(), - ); - let running_source = Arc::new( - MemoryExec::try_new(&[input1.clone()], schema.clone(), None) + let usual_source = MemorySourceConfig::try_new_exec( + &[vec![concat_input_record]], + schema.clone(), + None, + ) + .unwrap(); + + let running_source = Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None) .unwrap() .try_with_sort_information(vec![sort_keys]) .unwrap(), - ); + ))); let aggregate_expr = vec![ diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 41c12193f018..8e8178e55d87 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -15,21 +15,16 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; +use std::time::SystemTime; + +use crate::fuzz_cases::join_fuzz::JoinTestType::{HjSmj, NljHj}; + use arrow::array::{ArrayRef, Int32Array}; use arrow::compute::SortOptions; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; use arrow_schema::Schema; -use std::sync::Arc; -use std::time::SystemTime; - -use datafusion_common::ScalarValue; -use datafusion_physical_expr::expressions::Literal; -use datafusion_physical_expr::PhysicalExprRef; - -use itertools::Itertools; -use rand::Rng; - use datafusion::common::JoinSide; use datafusion::logical_expr::{JoinType, Operator}; use datafusion::physical_expr::expressions::BinaryExpr; @@ -39,10 +34,15 @@ use datafusion::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; use datafusion::physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, SortMergeJoinExec, }; -use datafusion::physical_plan::memory::MemoryExec; - -use crate::fuzz_cases::join_fuzz::JoinTestType::{HjSmj, NljHj}; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_common::ScalarValue; +use datafusion_physical_expr::expressions::Literal; +use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; + +use itertools::Itertools; +use rand::Rng; use test_utils::stagger_batch_with_seed; // Determines what Fuzz tests needs to run @@ -425,13 +425,15 @@ impl JoinFuzzTestCase { column_indices } - fn left_right(&self) -> (Arc, Arc) { + fn left_right(&self) -> (Arc, Arc) { let schema1 = self.input1[0].schema(); let schema2 = self.input2[0].schema(); let left = - Arc::new(MemoryExec::try_new(&[self.input1.clone()], schema1, None).unwrap()); + MemorySourceConfig::try_new_exec(&[self.input1.clone()], schema1, None) + .unwrap(); let right = - Arc::new(MemoryExec::try_new(&[self.input2.clone()], schema2, None).unwrap()); + MemorySourceConfig::try_new_exec(&[self.input2.clone()], schema2, None) + .unwrap(); (left, right) } diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 4e895920dd3d..35fca789ddcb 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -27,11 +27,12 @@ use arrow::{ use datafusion::physical_plan::{ collect, expressions::{col, PhysicalSortExpr}, - memory::MemoryExec, + memory::MemorySourceConfig, sorts::sort_preserving_merge::SortPreservingMergeExec, }; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_physical_expr_common::sort_expr::LexOrdering; + use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; #[tokio::test] @@ -116,8 +117,8 @@ async fn run_merge_test(input: Vec>) { }, }]); - let exec = MemoryExec::try_new(&input, schema, None).unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec(&input, schema, None).unwrap(); + let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let session_config = SessionConfig::new().with_batch_size(batch_size); let ctx = SessionContext::new_with_config(session_config); diff --git a/datafusion/core/tests/fuzz_cases/pruning.rs b/datafusion/core/tests/fuzz_cases/pruning.rs index 8ce980ee080b..a43886b6df21 100644 --- a/datafusion/core/tests/fuzz_cases/pruning.rs +++ b/datafusion/core/tests/fuzz_cases/pruning.rs @@ -23,7 +23,7 @@ use bytes::{BufMut, Bytes, BytesMut}; use datafusion::{ datasource::{ listing::PartitionedFile, - physical_plan::{parquet::ParquetExecBuilder, FileScanConfig}, + physical_plan::{FileScanConfig, ParquetSource}, }, prelude::*, }; @@ -303,24 +303,25 @@ async fn execute_with_predicate( schema: Arc, ctx: &SessionContext, ) -> Vec { - let scan = - FileScanConfig::new(ObjectStoreUrl::parse("memory://").unwrap(), schema.clone()) - .with_file_group( - files - .iter() - .map(|test_file| { - PartitionedFile::new( - test_file.path.clone(), - test_file.size as u64, - ) - }) - .collect(), - ); - let mut builder = ParquetExecBuilder::new(scan); - if prune_stats { - builder = builder.with_predicate(predicate.clone()) - } - let exec = Arc::new(builder.build()) as Arc; + let parquet_source = if prune_stats { + ParquetSource::default().with_predicate(Arc::clone(&schema), predicate.clone()) + } else { + ParquetSource::default() + }; + let scan = FileScanConfig::new( + ObjectStoreUrl::parse("memory://").unwrap(), + schema.clone(), + Arc::new(parquet_source), + ) + .with_file_group( + files + .iter() + .map(|test_file| { + PartitionedFile::new(test_file.path.clone(), test_file.size as u64) + }) + .collect(), + ); + let exec = scan.new_exec(); let exec = Arc::new(FilterExec::try_new(predicate, exec).unwrap()) as Arc; diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 19ffa69f11d3..ecc077261acc 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -17,6 +17,8 @@ //! Fuzz Test for various corner cases sorting RecordBatches exceeds available memory and should spill +use std::sync::Arc; + use arrow::{ array::{ArrayRef, Int32Array}, compute::SortOptions, @@ -24,15 +26,15 @@ use arrow::{ }; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_plan::expressions::PhysicalSortExpr; -use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_execution::memory_pool::GreedyMemoryPool; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::memory::MemorySourceConfig; + use rand::Rng; -use std::sync::Arc; use test_utils::{batches_to_vec, partitions_to_sorted_vec}; const KB: usize = 1 << 10; @@ -123,8 +125,8 @@ impl SortTest { }, }]); - let exec = MemoryExec::try_new(&input, schema, None).unwrap(); - let sort = Arc::new(SortExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec(&input, schema, None).unwrap(); + let sort = Arc::new(SortExec::new(sort, exec)); let session_config = SessionConfig::new(); let session_ctx = if let Some(pool_size) = self.pool_size { diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index daa282c8fe4a..602205beadcc 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -25,7 +25,6 @@ mod sp_repartition_fuzz_tests { use datafusion::physical_plan::{ collect, - memory::MemoryExec, metrics::{BaselineMetrics, ExecutionPlanMetricsSet}, repartition::RepartitionExec, sorts::sort_preserving_merge::SortPreservingMergeExec, @@ -46,6 +45,8 @@ mod sp_repartition_fuzz_tests { use test_utils::add_empty_batches; use datafusion_physical_expr_common::sort_expr::LexOrdering; + use datafusion_physical_plan::memory::MemorySourceConfig; + use datafusion_physical_plan::source::DataSourceExec; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; @@ -323,23 +324,23 @@ mod sp_repartition_fuzz_tests { /// "SortPreservingMergeExec: [a@0 ASC,b@1 ASC,c@2 ASC]", /// " SortPreservingRepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 2), input_partitions=2", (Partitioning can be roundrobin also) /// " SortPreservingRepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 2), input_partitions=1", (Partitioning can be roundrobin also) - /// " MemoryExec: partitions=1, partition_sizes=[75]", + /// " DataSourceExec: partitions=1, partition_sizes=[75]", /// and / or /// "SortPreservingMergeExec: [a@0 ASC,b@1 ASC,c@2 ASC]", /// " SortPreservingRepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 2), input_partitions=2", (Partitioning can be roundrobin also) /// " RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 2), input_partitions=1", (Partitioning can be roundrobin also) - /// " MemoryExec: partitions=1, partition_sizes=[75]", + /// " DataSourceExec: partitions=1, partition_sizes=[75]", /// preserves ordering. Input fed to the plan above should be same with the output of the plan. async fn run_sort_preserving_repartition_test( input1: Vec, - // If `true`, first repartition executor after `MemoryExec` will be in `RoundRobin` mode + // If `true`, first repartition executor after `DataSourceExec` will be in `RoundRobin` mode // else it will be in `Hash` mode is_first_roundrobin: bool, - // If `true`, first repartition executor after `MemoryExec` will be `SortPreservingRepartitionExec` - // If `false`, first repartition executor after `MemoryExec` will be `RepartitionExec` (Since its input + // If `true`, first repartition executor after `DataSourceExec` will be `SortPreservingRepartitionExec` + // If `false`, first repartition executor after `DataSourceExec` will be `RepartitionExec` (Since its input // partition number is 1, `RepartitionExec` also preserves ordering.). is_first_sort_preserving: bool, - // If `true`, second repartition executor after `MemoryExec` will be in `RoundRobin` mode + // If `true`, second repartition executor after `DataSourceExec` will be in `RoundRobin` mode // else it will be in `Hash` mode is_second_roundrobin: bool, ) { @@ -357,11 +358,12 @@ mod sp_repartition_fuzz_tests { let concat_input_record = concat_batches(&schema, &input1).unwrap(); let running_source = Arc::new( - MemoryExec::try_new(&[input1.clone()], schema.clone(), None) + MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None) .unwrap() .try_with_sort_information(vec![sort_keys.clone()]) .unwrap(), ); + let running_source = Arc::new(DataSourceExec::new(running_source)); let hash_exprs = vec![col("c", &schema).unwrap()]; let intermediate = match (is_first_roundrobin, is_first_sort_preserving) { diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 979aa5a2da03..9c66bf2d78f2 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -22,7 +22,7 @@ use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; -use datafusion::physical_plan::memory::MemoryExec; +use datafusion::functions_window::row_number::row_number_udwf; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ create_window_expr, schema_add_window_field, BoundedWindowAggExec, WindowAggExec, @@ -30,6 +30,7 @@ use datafusion::physical_plan::windows::{ use datafusion::physical_plan::InputOrderMode::{Linear, PartiallySorted, Sorted}; use datafusion::physical_plan::{collect, InputOrderMode}; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_common::HashMap; use datafusion_common::{Result, ScalarValue}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::type_coercion::functions::data_types_with_aggregate_udf; @@ -39,21 +40,21 @@ use datafusion_expr::{ use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::min_max::{max_udaf, min_udaf}; use datafusion_functions_aggregate::sum::sum_udaf; -use datafusion_physical_expr::expressions::{cast, col, lit}; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use test_utils::add_empty_batches; - -use datafusion::functions_window::row_number::row_number_udwf; -use datafusion_common::HashMap; use datafusion_functions_window::lead_lag::{lag_udwf, lead_udwf}; use datafusion_functions_window::nth_value::{ first_value_udwf, last_value_udwf, nth_value_udwf, }; use datafusion_functions_window::rank::{dense_rank_udwf, rank_udwf}; +use datafusion_physical_expr::expressions::{cast, col, lit}; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; + use rand::distributions::Alphanumeric; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; +use test_utils::add_empty_batches; #[tokio::test(flavor = "multi_thread", worker_threads = 16)] async fn window_bounded_window_random_comparison() -> Result<()> { @@ -64,23 +65,23 @@ async fn window_bounded_window_random_comparison() -> Result<()> { // In sorted mode physical plans are in the form for WindowAggExec //``` // WindowAggExec - // MemoryExec] + // DataSourceExec] // ``` // and in the form for BoundedWindowAggExec // ``` // BoundedWindowAggExec - // MemoryExec + // DataSourceExec // ``` // In Linear and PartiallySorted mode physical plans are in the form for WindowAggExec //``` // WindowAggExec // SortExec(required by window function) - // MemoryExec] + // DataSourceExec] // ``` // and in the form for BoundedWindowAggExec // ``` // BoundedWindowAggExec - // MemoryExec + // DataSourceExec // ``` let test_cases = vec![ (vec!["a"], vec!["a"], Sorted), @@ -159,11 +160,8 @@ async fn bounded_window_causal_non_causal() -> Result<()> { // Remove empty batches: batches.retain(|batch| batch.num_rows() > 0); let schema = batches[0].schema(); - let memory_exec = Arc::new(MemoryExec::try_new( - &[batches.clone()], - schema.clone(), - None, - )?); + let memory_exec = + MemorySourceConfig::try_new_exec(&[batches.clone()], schema.clone(), None)?; // Different window functions to test causality let window_functions = vec![ @@ -638,10 +636,10 @@ async fn run_window_test( options: Default::default(), }, ]); - let mut exec1 = Arc::new( - MemoryExec::try_new(&[vec![concat_input_record]], schema.clone(), None)? + let mut exec1 = Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&[vec![concat_input_record]], schema.clone(), None)? .try_with_sort_information(vec![source_sort_keys.clone()])?, - ) as _; + ))) as _; // Table is ordered according to ORDER BY a, b, c In linear test we use PARTITION BY b, ORDER BY a // For WindowAggExec to produce correct result it need table to be ordered by b,a. Hence add a sort. if is_linear { @@ -664,10 +662,10 @@ async fn run_window_test( exec1, vec![], )?) as _; - let exec2 = Arc::new( - MemoryExec::try_new(&[input1.clone()], schema.clone(), None)? + let exec2 = Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None)? .try_with_sort_information(vec![source_sort_keys.clone()])?, - ); + ))); let running_window_exec = Arc::new(BoundedWindowAggExec::try_new( vec![create_window_expr( &window_fn, diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 212ffdaaa2a5..733d6cdee0ea 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -32,7 +32,6 @@ use datafusion::datasource::{MemTable, TableProvider}; use datafusion::execution::disk_manager::DiskManagerConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::execution::session_state::SessionStateBuilder; -use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::streaming::PartitionStream; use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; @@ -48,6 +47,8 @@ use datafusion_expr::{Expr, TableType}; use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_optimizer::join_selection::JoinSelection; use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::spill::get_record_batch_memory_size; use test_utils::AccessLogGenerator; @@ -241,15 +242,15 @@ async fn sort_preserving_merge() { // SortPreservingMergeExec (not a Sort which would compete // with the SortPreservingMergeExec for memory) &[ - "+---------------+------------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+------------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b ASC NULLS LAST, fetch=10 |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 |", - "| | MemoryExec: partitions=2, partition_sizes=[5, 5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", - "| | |", - "+---------------+------------------------------------------------------------------------------------------------------------+", + "+---------------+--------------------------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+--------------------------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b ASC NULLS LAST, fetch=10 |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 |", + "| | DataSourceExec: partitions=2, partition_sizes=[5, 5], fetch=10, output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+--------------------------------------------------------------------------------------------------------------------------+" ] ) .run() @@ -288,15 +289,15 @@ async fn sort_spill_reservation() { // also merge, so we can ensure the sort could finish // given enough merging memory &[ - "+---------------+---------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+---------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] |", - "| | MemoryExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", - "| | |", - "+---------------+---------------------------------------------------------------------------------------------------------+", + "+---------------+-------------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+-------------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] |", + "| | DataSourceExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+-------------------------------------------------------------------------------------------------------------+", ] ); @@ -844,10 +845,13 @@ impl TableProvider for SortedTableProvider { _filters: &[Expr], _limit: Option, ) -> Result> { - let mem_exec = - MemoryExec::try_new(&self.batches, self.schema(), projection.cloned())? - .try_with_sort_information(self.sort_information.clone())?; - - Ok(Arc::new(mem_exec)) + let mem_conf = MemorySourceConfig::try_new( + &self.batches, + self.schema(), + projection.cloned(), + )? + .try_with_sort_information(self.sort_information.clone())?; + + Ok(Arc::new(DataSourceExec::new(Arc::new(mem_conf)))) } } diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index dc57ba1e443a..928b650e0300 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -28,7 +28,7 @@ use datafusion::datasource::file_format::parquet::fetch_parquet_metadata; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{ - FileMeta, FileScanConfig, ParquetExec, ParquetFileMetrics, ParquetFileReaderFactory, + FileMeta, FileScanConfig, ParquetFileMetrics, ParquetFileReaderFactory, ParquetSource, }; use datafusion::physical_plan::collect; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; @@ -75,23 +75,26 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { }) .collect(); - // prepare the scan - let parquet_exec = ParquetExec::builder( - FileScanConfig::new( - // just any url that doesn't point to in memory object store - ObjectStoreUrl::local_filesystem(), - file_schema, - ) - .with_file_group(file_group), + let source = Arc::new( + ParquetSource::default() + // prepare the scan + .with_parquet_file_reader_factory(Arc::new( + InMemoryParquetFileReaderFactory(Arc::clone(&in_memory_object_store)), + )), + ); + let base_config = FileScanConfig::new( + // just any url that doesn't point to in memory object store + ObjectStoreUrl::local_filesystem(), + file_schema, + source, ) - .build() - .with_parquet_file_reader_factory(Arc::new(InMemoryParquetFileReaderFactory( - Arc::clone(&in_memory_object_store), - ))); + .with_file_group(file_group); + + let parquet_exec = base_config.new_exec(); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let read = collect(Arc::new(parquet_exec), task_ctx).await.unwrap(); + let read = collect(parquet_exec, task_ctx).await.unwrap(); let expected = [ "+-----+----+----+", diff --git a/datafusion/core/tests/parquet/external_access_plan.rs b/datafusion/core/tests/parquet/external_access_plan.rs index 61a9e9b5757c..216f03aac746 100644 --- a/datafusion/core/tests/parquet/external_access_plan.rs +++ b/datafusion/core/tests/parquet/external_access_plan.rs @@ -15,26 +15,30 @@ // specific language governing permissions and limitations // under the License. -//! Tests for passing user provided [`ParquetAccessPlan`]` to `ParquetExec`]` +//! Tests for passing user provided [`ParquetAccessPlan`]` to `DataSourceExec`]` + +use std::path::Path; +use std::sync::Arc; + use crate::parquet::utils::MetricsFinder; use crate::parquet::{create_data_batch, Scenario}; + use arrow::util::pretty::pretty_format_batches; use arrow_schema::SchemaRef; use datafusion::common::Result; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::parquet::{ParquetAccessPlan, RowGroupAccess}; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::prelude::SessionContext; use datafusion_common::{assert_contains, DFSchema}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::ExecutionPlan; + use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; -use std::path::Path; -use std::sync::Arc; use tempfile::NamedTempFile; #[tokio::test] @@ -274,7 +278,7 @@ struct Test { impl Test { /// Runs the test case, panic'ing on error. /// - /// Returns the [`MetricsSet`] from the [`ParquetExec`] + /// Returns the [`MetricsSet`] from the [`DataSourceExec`] async fn run_success(self) -> MetricsSet { let Self { access_plan, @@ -334,23 +338,22 @@ impl TestFull { partitioned_file = partitioned_file.with_extensions(Arc::new(access_plan)); } - // Create a ParquetExec to read the file + // Create a DataSourceExec to read the file let object_store_url = ObjectStoreUrl::local_filesystem(); - let config = FileScanConfig::new(object_store_url, schema.clone()) - .with_file(partitioned_file); - - let mut builder = ParquetExec::builder(config); - // add the predicate, if requested - if let Some(predicate) = predicate { + let source = if let Some(predicate) = predicate { let df_schema = DFSchema::try_from(schema.clone())?; let predicate = ctx.create_physical_expr(predicate, &df_schema)?; - builder = builder.with_predicate(predicate); - } + Arc::new(ParquetSource::default().with_predicate(schema.clone(), predicate)) + } else { + Arc::new(ParquetSource::default()) + }; + let config = FileScanConfig::new(object_store_url, schema.clone(), source) + .with_file(partitioned_file); - let plan: Arc = builder.build_arc(); + let plan: Arc = config.new_exec(); - // run the ParquetExec and collect the results + // run the DataSourceExec and collect the results let results = datafusion::physical_plan::collect(Arc::clone(&plan), ctx.task_ctx()).await?; diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 4b5d22bfa71f..82024a731ed3 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -22,9 +22,9 @@ use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, }; -use datafusion::datasource::physical_plan::ParquetExec; use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; +use datafusion::execution::session_state::SessionStateBuilder; use datafusion::prelude::SessionContext; use datafusion_common::stats::Precision; use datafusion_execution::cache::cache_manager::CacheManagerConfig; @@ -33,9 +33,10 @@ use datafusion_execution::cache::cache_unit::{ }; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; - -use datafusion::execution::session_state::SessionStateBuilder; use datafusion_expr::{col, lit, Expr}; +use datafusion_physical_plan::source::DataSourceExec; + +use datafusion::datasource::physical_plan::FileScanConfig; use tempfile::tempdir; #[tokio::test] @@ -149,10 +150,12 @@ async fn list_files_with_session_level_cache() { //Session 1 first time list files assert_eq!(get_list_file_cache_size(&state1), 0); let exec1 = table1.scan(&state1, None, &[], None).await.unwrap(); - let parquet1 = exec1.as_any().downcast_ref::().unwrap(); + let data_source = exec1.as_any().downcast_ref::().unwrap(); + let source = data_source.source(); + let parquet1 = source.as_any().downcast_ref::().unwrap(); assert_eq!(get_list_file_cache_size(&state1), 1); - let fg = &parquet1.base_config().file_groups; + let fg = &parquet1.file_groups; assert_eq!(fg.len(), 1); assert_eq!(fg.first().unwrap().len(), 1); @@ -160,10 +163,12 @@ async fn list_files_with_session_level_cache() { //check session 1 cache result not show in session 2 assert_eq!(get_list_file_cache_size(&state2), 0); let exec2 = table2.scan(&state2, None, &[], None).await.unwrap(); - let parquet2 = exec2.as_any().downcast_ref::().unwrap(); + let data_source = exec2.as_any().downcast_ref::().unwrap(); + let source = data_source.source(); + let parquet2 = source.as_any().downcast_ref::().unwrap(); assert_eq!(get_list_file_cache_size(&state2), 1); - let fg2 = &parquet2.base_config().file_groups; + let fg2 = &parquet2.file_groups; assert_eq!(fg2.len(), 1); assert_eq!(fg2.first().unwrap().len(), 1); @@ -171,10 +176,12 @@ async fn list_files_with_session_level_cache() { //check session 1 cache result not show in session 2 assert_eq!(get_list_file_cache_size(&state1), 1); let exec3 = table1.scan(&state1, None, &[], None).await.unwrap(); - let parquet3 = exec3.as_any().downcast_ref::().unwrap(); + let data_source = exec3.as_any().downcast_ref::().unwrap(); + let source = data_source.source(); + let parquet3 = source.as_any().downcast_ref::().unwrap(); assert_eq!(get_list_file_cache_size(&state1), 1); - let fg = &parquet3.base_config().file_groups; + let fg = &parquet3.file_groups; assert_eq!(fg.len(), 1); assert_eq!(fg.first().unwrap().len(), 1); // List same file no increase diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 65bfd0340125..ea86bf3685bb 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use crate::parquet::Unit::Page; use crate::parquet::{ContextWithParquet, Scenario}; @@ -22,7 +24,7 @@ use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::file_format::FileFormat; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::execution::context::SessionState; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; @@ -31,12 +33,13 @@ use datafusion_common::{ScalarValue, ToDFSchema}; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; +use datafusion_physical_plan::source::DataSourceExec; use futures::StreamExt; use object_store::path::Path; use object_store::ObjectMeta; -async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { +async fn get_parquet_exec(state: &SessionState, filter: Expr) -> DataSourceExec { let object_store_url = ObjectStoreUrl::local_filesystem(); let store = state.runtime_env().object_store(&object_store_url).unwrap(); @@ -71,12 +74,15 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { let execution_props = ExecutionProps::new(); let predicate = create_physical_expr(&filter, &df_schema, &execution_props).unwrap(); - ParquetExec::builder( - FileScanConfig::new(object_store_url, schema).with_file(partitioned_file), - ) - .with_predicate(predicate) - .build() - .with_enable_page_index(true) + let source = Arc::new( + ParquetSource::default() + .with_predicate(Arc::clone(&schema), predicate) + .with_enable_page_index(true), + ); + let base_config = + FileScanConfig::new(object_store_url, schema, source).with_file(partitioned_file); + + DataSourceExec::new(Arc::new(base_config)) } #[tokio::test] diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index af9411f40ecb..3b9c43685deb 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -23,7 +23,7 @@ use arrow_array::types::Int32Type; use arrow_array::{ArrayRef, DictionaryArray, Float32Array, Int64Array, StringArray}; use arrow_schema::DataType; use datafusion::assert_batches_sorted_eq; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::physical_plan::collect; use datafusion::prelude::SessionContext; use datafusion_common::Result; @@ -59,15 +59,16 @@ async fn multi_parquet_coercion() { Field::new("c2", DataType::Int32, true), Field::new("c3", DataType::Float64, true), ])); - let parquet_exec = ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) - .with_file_group(file_group), - ) - .build(); + let source = Arc::new(ParquetSource::default()); + let conf = + FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema, source) + .with_file_group(file_group); + + let parquet_exec = conf.new_exec(); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let read = collect(Arc::new(parquet_exec), task_ctx).await.unwrap(); + let read = collect(parquet_exec, task_ctx).await.unwrap(); let expected = [ "+-------+----+------+", @@ -113,16 +114,18 @@ async fn multi_parquet_coercion_projection() { Field::new("c2", DataType::Int32, true), Field::new("c3", DataType::Float64, true), ])); - let parquet_exec = ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) - .with_file_group(file_group) - .with_projection(Some(vec![1, 0, 2])), + let parquet_exec = FileScanConfig::new( + ObjectStoreUrl::local_filesystem(), + file_schema, + Arc::new(ParquetSource::default()), ) - .build(); + .with_file_group(file_group) + .with_projection(Some(vec![1, 0, 2])) + .new_exec(); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let read = collect(Arc::new(parquet_exec), task_ctx).await.unwrap(); + let read = collect(parquet_exec, task_ctx).await.unwrap(); let expected = [ "+----+-------+------+", diff --git a/datafusion/core/tests/parquet/utils.rs b/datafusion/core/tests/parquet/utils.rs index d8d2b2fbb8a5..dd5541461ff6 100644 --- a/datafusion/core/tests/parquet/utils.rs +++ b/datafusion/core/tests/parquet/utils.rs @@ -17,11 +17,12 @@ //! Utilities for parquet tests -use datafusion::datasource::physical_plan::ParquetExec; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion_physical_plan::metrics::MetricsSet; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{accept, ExecutionPlan, ExecutionPlanVisitor}; -/// Find the metrics from the first ParquetExec encountered in the plan +/// Find the metrics from the first DataSourceExec encountered in the plan #[derive(Debug)] pub struct MetricsFinder { metrics: Option, @@ -46,8 +47,18 @@ impl MetricsFinder { impl ExecutionPlanVisitor for MetricsFinder { type Error = std::convert::Infallible; fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { - if plan.as_any().downcast_ref::().is_some() { - self.metrics = plan.metrics(); + if let Some(exec) = plan.as_any().downcast_ref::() { + let source = exec.source(); + if let Some(file_config) = source.as_any().downcast_ref::() { + if file_config + .file_source() + .as_any() + .downcast_ref::() + .is_some() + { + self.metrics = exec.metrics(); + } + } } // stop searching once we have found the metrics Ok(self.metrics.is_none()) diff --git a/datafusion/core/tests/parquet_exec.rs b/datafusion/core/tests/parquet_config.rs similarity index 93% rename from datafusion/core/tests/parquet_exec.rs rename to datafusion/core/tests/parquet_config.rs index f41f82a76c67..61d67cbdc49e 100644 --- a/datafusion/core/tests/parquet_exec.rs +++ b/datafusion/core/tests/parquet_config.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! End to end test for `ParquetExec` and related components +//! End to end test for `ParquetSource` and related components /// Run all tests that are found in the `parquet` directory mod parquet; diff --git a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs index f5ecd41ab11e..1757c7150bfe 100644 --- a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs @@ -36,12 +36,13 @@ use datafusion_physical_plan::aggregates::PhysicalGroupBy; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::common; use datafusion_physical_plan::filter::FilterExec; -use datafusion_physical_plan::memory::MemoryExec; +use datafusion_physical_plan::memory::MemorySourceConfig; use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::ExecutionPlan; -/// Mock data using a MemoryExec which has an exact count statistic -fn mock_data() -> Result> { +/// Mock data using a MemorySourceConfig which has an exact count statistic +fn mock_data() -> Result> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), @@ -55,11 +56,7 @@ fn mock_data() -> Result> { ], )?; - Ok(Arc::new(MemoryExec::try_new( - &[vec![batch]], - Arc::clone(&schema), - None, - )?)) + MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) } /// Checks that the count optimization was applied and we still get the right result diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index f0588e45cc6a..568be0d18f24 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -18,7 +18,7 @@ //! Tests for [`CombinePartialFinalAggregate`] physical optimizer rule //! //! Note these tests are not in the same module as the optimizer pass because -//! they rely on `ParquetExec` which is in the core crate. +//! they rely on `DataSourceExec` which is in the core crate. use std::sync::Arc; @@ -148,7 +148,7 @@ fn aggregations_not_combined() -> datafusion_common::Result<()> { "AggregateExec: mode=Final, gby=[], aggr=[COUNT(1)]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c], file_type=parquet", ]; assert_optimized!(expected, plan); @@ -168,7 +168,7 @@ fn aggregations_not_combined() -> datafusion_common::Result<()> { let expected = &[ "AggregateExec: mode=Final, gby=[], aggr=[COUNT(2)]", "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c], file_type=parquet", ]; assert_optimized!(expected, plan); @@ -193,7 +193,7 @@ fn aggregations_combined() -> datafusion_common::Result<()> { // should combine the Partial/Final AggregateExecs to the Single AggregateExec let expected = &[ "AggregateExec: mode=Single, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c], file_type=parquet", ]; assert_optimized!(expected, plan); @@ -229,7 +229,7 @@ fn aggregations_with_group_combined() -> datafusion_common::Result<()> { // should combine the Partial/Final AggregateExecs to the Single AggregateExec let expected = &[ "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[Sum(b)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c], file_type=parquet", ]; assert_optimized!(expected, plan); @@ -273,7 +273,7 @@ fn aggregations_with_limit_combined() -> datafusion_common::Result<()> { // with the final limit preserved let expected = &[ "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[], lim=[5]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c], file_type=parquet", ]; assert_optimized!(expected, plan); diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 856f7dc8e8a9..855550dc748a 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -30,7 +30,7 @@ use datafusion::config::ConfigOptions; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig, ParquetSource}; use datafusion_common::error::Result; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; @@ -57,6 +57,7 @@ use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_plan::PlanProperties; @@ -160,73 +161,63 @@ impl ExecutionPlan for SortRequiredExec { } } -fn parquet_exec() -> Arc { +fn parquet_exec() -> Arc { parquet_exec_with_sort(vec![]) } -fn parquet_exec_multiple() -> Arc { +fn parquet_exec_multiple() -> Arc { parquet_exec_multiple_sorted(vec![]) } /// Created a sorted parquet exec with multiple files -fn parquet_exec_multiple_sorted(output_ordering: Vec) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file_groups(vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ]) - .with_output_ordering(output_ordering), +fn parquet_exec_multiple_sorted( + output_ordering: Vec, +) -> Arc { + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(ParquetSource::default()), ) - .build_arc() + .with_file_groups(vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ]) + .with_output_ordering(output_ordering) + .new_exec() } -fn csv_exec() -> Arc { +fn csv_exec() -> Arc { csv_exec_with_sort(vec![]) } -fn csv_exec_with_sort(output_ordering: Vec) -> Arc { - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), +fn csv_exec_with_sort(output_ordering: Vec) -> Arc { + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering) + .new_exec() } -fn csv_exec_multiple() -> Arc { +fn csv_exec_multiple() -> Arc { csv_exec_multiple_sorted(vec![]) } // Created a sorted parquet exec with multiple files -fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file_groups(vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ]) - .with_output_ordering(output_ordering), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), +fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), ) + .with_file_groups(vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ]) + .with_output_ordering(output_ordering) + .new_exec() } fn projection_exec_with_alias( @@ -576,14 +567,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 4 RepartitionExecs _ => vec![ @@ -592,14 +583,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true); @@ -639,14 +630,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 4 RepartitionExecs _ => @@ -656,14 +647,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true); @@ -715,13 +706,13 @@ fn multi_joins_after_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); assert_optimized!(expected, top_join, false); @@ -741,13 +732,13 @@ fn multi_joins_after_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); assert_optimized!(expected, top_join, false); @@ -794,13 +785,13 @@ fn multi_joins_after_multi_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); @@ -836,12 +827,12 @@ fn join_after_agg_alias() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true); assert_optimized!(expected, join, false); @@ -889,12 +880,12 @@ fn hash_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true); assert_optimized!(expected, join, false); @@ -1008,19 +999,19 @@ fn multi_hash_join_key_ordering() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, filter_top_join.clone(), true); assert_optimized!(expected, filter_top_join, false); @@ -1146,19 +1137,19 @@ fn reorder_join_keys_to_left_input() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", "RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, reordered); @@ -1280,19 +1271,19 @@ fn reorder_join_keys_to_right_input() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", "RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, reordered); @@ -1353,16 +1344,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs // Since ordering of the left child is not preserved after SortMergeJoin @@ -1382,16 +1373,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true, true); @@ -1405,16 +1396,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs // Since ordering of the left child is not preserved after SortMergeJoin @@ -1436,16 +1427,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected_first_sort_enforcement, top_join, false, true); @@ -1471,16 +1462,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ @@ -1491,16 +1482,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // this match arm cannot be reached _ => unreachable!() @@ -1515,16 +1506,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ @@ -1537,16 +1528,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // this match arm cannot be reached _ => unreachable!() @@ -1616,14 +1607,14 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true, true); @@ -1639,7 +1630,7 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", @@ -1649,7 +1640,7 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, join, false, true); @@ -1680,7 +1671,7 @@ fn merge_does_not_need_sort() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", "CoalesceBatchesExec: target_batch_size=4096", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, exec, true); @@ -1692,7 +1683,7 @@ fn merge_does_not_need_sort() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "CoalesceBatchesExec: target_batch_size=4096", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, exec, false); @@ -1728,12 +1719,12 @@ fn union_to_interleave() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan.clone(), false); @@ -1771,12 +1762,12 @@ fn union_not_to_interleave() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; // no sort in the plan but since we need it as a parameter, make it default false let prefer_existing_sort = false; @@ -1811,7 +1802,7 @@ fn added_repartition_to_single_partition() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1830,7 +1821,7 @@ fn repartition_deepest_node() -> Result<()> { "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1850,7 +1841,7 @@ fn repartition_unsorted_limit() -> Result<()> { "FilterExec: c@2 = 0", // nothing sorts the data, so the local limit doesn't require sorted data either "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1873,7 +1864,7 @@ fn repartition_sorted_limit() -> Result<()> { "LocalLimitExec: fetch=100", // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1900,7 +1891,7 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { // is still satisfied. "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1931,7 +1922,7 @@ fn repartition_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", // Expect no repartition to happen for local limit - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1945,12 +1936,12 @@ fn repartition_ignores_union() -> Result<()> { let expected = &[ "UnionExec", - // Expect no repartition of ParquetExec - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + // Expect no repartition of DataSourceExec + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1972,7 +1963,7 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { // need resort as the data was not sorted correctly let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1997,7 +1988,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2005,7 +1996,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan, false); @@ -2027,8 +2018,8 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2037,8 +2028,8 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan, false); @@ -2064,7 +2055,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { "SortRequiredExec: [d@3 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true, true); @@ -2102,11 +2093,11 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { "UnionExec", // union input 1: no repartitioning "SortRequiredExec: [c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", // union input 2: should repartition "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2139,7 +2130,7 @@ fn repartition_transitively_with_projection() -> Result<()> { // Since this projection is not trivial, increasing parallelism is beneficial "ProjectionExec: expr=[a@0 + b@1 as sum]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2150,7 +2141,7 @@ fn repartition_transitively_with_projection() -> Result<()> { // Since this projection is not trivial, increasing parallelism is beneficial "ProjectionExec: expr=[a@0 + b@1 as sum]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2182,7 +2173,7 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { "SortRequiredExec: [c@2 ASC]", // Since this projection is trivial, increasing parallelism is not beneficial "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2215,7 +2206,7 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", // Since this projection is trivial, increasing parallelism is not beneficial "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2238,7 +2229,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { // Expect repartition on the input to the sort (as it can benefit from additional parallelism) "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2249,7 +2240,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { "FilterExec: c@2 = 0", // Expect repartition on the input of the filter (as it can benefit from additional parallelism) "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2285,7 +2276,7 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> "FilterExec: c@2 = 0", // repartition is lowest down "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2296,7 +2287,7 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2313,13 +2304,13 @@ fn parallelization_single_partition() -> Result<()> { "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "ParquetExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); @@ -2344,7 +2335,7 @@ fn parallelization_multiple_files() -> Result<()> { let expected = [ "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; + "DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let target_partitions = 3; let repartition_size = 1; assert_optimized!( @@ -2361,7 +2352,7 @@ fn parallelization_multiple_files() -> Result<()> { let expected = [ "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let target_partitions = 8; let repartition_size = 1; @@ -2380,7 +2371,7 @@ fn parallelization_multiple_files() -> Result<()> { } #[test] -/// CsvExec on compressed csv file will not be partitioned +/// DataSourceExec on compressed csv file will not be partitioned /// (Not able to decompress chunked csv file) fn parallelization_compressed_csv() -> Result<()> { let compression_types = [ @@ -2396,14 +2387,14 @@ fn parallelization_compressed_csv() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; let expected_partitioned = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; for compression_type in compression_types { @@ -2414,23 +2405,14 @@ fn parallelization_compressed_csv() -> Result<()> { }; let plan = aggregate_exec_with_alias( - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(compression_type) - .build(), - ), + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_file_compression_type(compression_type) + .new_exec(), vec![("a".to_string(), "a".to_string())], ); assert_optimized!(expected, plan, true, false, 2, true, 10, false); @@ -2449,14 +2431,14 @@ fn parallelization_two_partitions() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); @@ -2474,14 +2456,14 @@ fn parallelization_two_partitions_into_four() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions - "ParquetExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions - "CsvExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); @@ -2505,7 +2487,7 @@ fn parallelization_sorted_limit() -> Result<()> { // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Doesn't parallelize for SortExec without preserve_partitioning - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -2513,7 +2495,7 @@ fn parallelization_sorted_limit() -> Result<()> { // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Doesn't parallelize for SortExec without preserve_partitioning - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2545,7 +2527,7 @@ fn parallelization_limit_with_filter() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // SortExec doesn't benefit from input partitioning - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -2557,7 +2539,7 @@ fn parallelization_limit_with_filter() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // SortExec doesn't benefit from input partitioning - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2589,7 +2571,7 @@ fn parallelization_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism "LocalLimitExec: fetch=100", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -2605,7 +2587,7 @@ fn parallelization_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism "LocalLimitExec: fetch=100", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2621,20 +2603,20 @@ fn parallelization_union_inputs() -> Result<()> { let expected_parquet = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2659,10 +2641,10 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // parallelization is not beneficial for SortPreservingMerge let expected_parquet = &[ - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let expected_csv = &[ - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2689,14 +2671,14 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let expected_parquet = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let expected_csv = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2723,11 +2705,11 @@ fn parallelization_does_not_benefit() -> Result<()> { // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism let expected_parquet = &[ "SortRequiredExec: [c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let expected_csv = &[ "SortRequiredExec: [c@2 ASC]", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2760,14 +2742,14 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; plans_matches_expected!(expected, &plan_parquet); // data should not be repartitioned / resorted let expected_parquet = &[ "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected_parquet, plan_parquet, true); @@ -2799,14 +2781,14 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; plans_matches_expected!(expected, &plan_csv); // data should not be repartitioned / resorted let expected_csv = &[ "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_csv, plan_csv, true); @@ -2823,14 +2805,14 @@ fn remove_redundant_roundrobins() -> Result<()> { " FilterExec: c@2 = 0", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; plans_matches_expected!(expected, &physical_plan); let expected = &[ "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2854,7 +2836,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { "CoalescePartitionsExec", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); @@ -2877,7 +2859,7 @@ fn preserve_ordering_through_repartition() -> Result<()> { "SortPreservingMergeExec: [d@3 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); @@ -2901,7 +2883,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); @@ -2911,7 +2893,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { "CoalescePartitionsExec", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan, false); @@ -2934,7 +2916,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { // Since after this stage c is constant. c@2 ASC ordering is already satisfied. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2962,7 +2944,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); @@ -2973,7 +2955,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan, false); @@ -2993,7 +2975,7 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let expected = &[ "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -3015,7 +2997,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { // by existing ordering at the source. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, physical_plan); @@ -3025,7 +3007,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { // EnforceDistribution rule doesn't satisfy this requirement either. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let mut config = ConfigOptions::new(); @@ -3053,7 +3035,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { // by existing ordering at the source. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_plan_txt!(expected, physical_plan); @@ -3062,7 +3044,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { // EnforceDistribution rule satisfy this requirement also. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let mut config = ConfigOptions::new(); @@ -3089,7 +3071,7 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is 1. In this case hash repartition is unnecessary assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); @@ -3119,7 +3101,7 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is larger than 2 (e.g partition number at the source). assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); @@ -3134,12 +3116,12 @@ fn optimize_away_unnecessary_repartition() -> Result<()> { let expected = &[ "CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; plans_matches_expected!(expected, physical_plan.clone()); let expected = - &["ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]"]; + &["DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet"]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -3157,7 +3139,7 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { " CoalescePartitionsExec", " FilterExec: c@2 = 0", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; plans_matches_expected!(expected, physical_plan.clone()); @@ -3165,7 +3147,7 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { "FilterExec: c@2 = 0", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index ab90eab74d3f..473346684b94 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -46,10 +46,9 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; -use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig, ParquetSource}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::sorts::sort::SortExec; @@ -63,24 +62,14 @@ fn csv_exec_ordered( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("file_path".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .with_has_header(true) - .with_delimeter(0) - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + Arc::new(CsvSource::new(true, 0, b'"')), ) + .with_file(PartitionedFile::new("file_path".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]) + .new_exec() } /// Created a sorted parquet exec @@ -90,12 +79,15 @@ pub fn parquet_exec_sorted( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), + let source = Arc::new(ParquetSource::default()); + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + source, ) - .build_arc() + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]) + .new_exec() } /// Create a sorted Csv exec @@ -105,24 +97,14 @@ fn csv_exec_sorted( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .with_has_header(false) - .with_delimeter(0) - .with_quote(0) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + Arc::new(CsvSource::new(false, 0, 0)), ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]) + .new_exec() } /// Runs the sort enforcement optimizer and asserts the plan @@ -229,12 +211,12 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { let expected_input = ["SortExec: expr=[a@2 ASC], preserve_partitioning=[false]", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet"]; let expected_optimized = ["HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -273,11 +255,11 @@ async fn test_bounded_window_set_monotonic_no_partition() -> Result<()> { "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -324,13 +306,13 @@ async fn test_bounded_plain_window_set_monotonic_with_partitions() -> Result<()> "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; let expected_optimized = [ "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -387,12 +369,12 @@ async fn test_bounded_plain_window_set_monotonic_with_partitions_partial() -> Re "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -430,12 +412,12 @@ async fn test_bounded_window_non_set_monotonic_sort() -> Result<()> { "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet", ]; let expected_optimized = [ "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -464,22 +446,22 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // We should keep the bottom `SortExec`. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -502,9 +484,9 @@ async fn test_union_inputs_sorted() -> Result<()> { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); @@ -534,9 +516,9 @@ async fn test_union_inputs_different_sorted() -> Result<()> { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); @@ -563,20 +545,20 @@ async fn test_union_inputs_different_sorted2() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_exprs, union); // Input is an invalid plan. In this case rule should add required sorting in appropriate places. - // First ParquetExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the + // First DataSourceExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the // required ordering of SortPreservingMergeExec. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -607,18 +589,18 @@ async fn test_union_inputs_different_sorted3() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // should adjust sorting in the first input of the union such that it is not unnecessarily fine let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -649,18 +631,18 @@ async fn test_union_inputs_different_sorted4() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -699,15 +681,15 @@ async fn test_union_inputs_different_sorted5() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -742,21 +724,21 @@ async fn test_union_inputs_different_sorted6() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // Should adjust the requirement in the third input of the union so // that it is not unnecessarily fine. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -782,16 +764,16 @@ async fn test_union_inputs_different_sorted7() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec let expected_output = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_output, physical_plan, true); Ok(()) @@ -833,14 +815,14 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { // example below. let expected_input = ["UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // Since `UnionExec` doesn't preserve ordering in the plan above. // We shouldn't keep SortExecs in the plan. let expected_optimized = ["UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -882,15 +864,15 @@ async fn test_window_multi_path_sort() -> Result<()> { " SortPreservingMergeExec: [nullable_col@0 DESC NULLS LAST]", " UnionExec", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; let expected_optimized = [ "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -920,14 +902,14 @@ async fn test_window_multi_path_sort2() -> Result<()> { " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -967,19 +949,19 @@ async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1024,8 +1006,8 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { ); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", join_plan2.as_str(), - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Left @@ -1035,9 +1017,9 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { vec![ join_plan.as_str(), " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ] } _ => { @@ -1046,9 +1028,9 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ] } }; @@ -1099,17 +1081,17 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { ); let expected_input = [spm_plan, join_plan2.as_str(), - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Right | JoinType::RightAnti => { // can push down the sort requirements and save 1 SortExec vec![ join_plan.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ] } _ => { @@ -1118,9 +1100,9 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ] } }; @@ -1154,16 +1136,16 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); // order by (nullable_col, col_b, col_a) @@ -1176,16 +1158,16 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1215,12 +1197,12 @@ async fn test_multilayer_coalesce_partitions() -> Result<()> { " FilterExec: NOT non_nullable_col@1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1243,12 +1225,12 @@ async fn test_with_lost_ordering_bounded() -> Result<()> { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1288,7 +1270,7 @@ async fn test_with_lost_ordering_unbounded_bounded( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -1305,14 +1287,14 @@ async fn test_with_lost_ordering_unbounded_bounded( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; let expected_optimized_bounded_parallelize_sort = vec![ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = if source_unbounded { @@ -1356,11 +1338,11 @@ async fn test_do_not_pushdown_through_spm() -> Result<()> { let expected_input = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; let expected_optimized = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1385,11 +1367,11 @@ async fn test_pushdown_through_spm() -> Result<()> { let expected_input = ["SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1413,7 +1395,7 @@ async fn test_window_multi_layer_requirement() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC, b@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", @@ -1421,7 +1403,7 @@ async fn test_window_multi_layer_requirement() -> Result<()> { " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -1443,7 +1425,7 @@ async fn test_not_replaced_with_partial_sort_for_bounded_input() -> Result<()> { ); let expected_input = [ "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC]" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC], file_type=parquet" ]; let expected_no_change = expected_input; assert_optimized!(expected_input, expected_no_change, physical_plan, false); @@ -1546,11 +1528,11 @@ async fn test_remove_unnecessary_sort() -> Result<()> { let expected_input = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1604,14 +1586,14 @@ async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " CoalesceBatchesExec: target_batch_size=128", " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", " FilterExec: NOT non_nullable_col@1", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " CoalesceBatchesExec: target_batch_size=128", " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1628,11 +1610,11 @@ async fn test_add_required_sort() -> Result<()> { let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1655,11 +1637,11 @@ async fn test_remove_unnecessary_sort1() -> Result<()> { " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1693,13 +1675,13 @@ async fn test_remove_unnecessary_sort2() -> Result<()> { " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1737,14 +1719,14 @@ async fn test_remove_unnecessary_sort3() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "AggregateExec: mode=Final, gby=[], aggr=[]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1788,18 +1770,18 @@ async fn test_remove_unnecessary_sort4() -> Result<()> { " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", " UnionExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " UnionExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1827,11 +1809,11 @@ async fn test_remove_unnecessary_sort6() -> Result<()> { let expected_input = [ "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1861,12 +1843,12 @@ async fn test_remove_unnecessary_sort7() -> Result<()> { let expected_input = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "GlobalLimitExec: skip=0, fetch=2", " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1894,12 +1876,12 @@ async fn test_remove_unnecessary_sort8() -> Result<()> { "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " LocalLimitExec: fetch=2", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "LocalLimitExec: fetch=2", " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1922,13 +1904,13 @@ async fn test_do_not_pushdown_through_limit() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1950,11 +1932,11 @@ async fn test_remove_unnecessary_spm1() -> Result<()> { "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1974,11 +1956,11 @@ async fn test_change_wrong_sorting() -> Result<()> { let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2001,11 +1983,11 @@ async fn test_change_wrong_sorting2() -> Result<()> { "SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2033,13 +2015,13 @@ async fn test_multiple_sort_window_exec() -> Result<()> { " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -2067,7 +2049,7 @@ async fn test_commutativity() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_eq!( expected_input, actual, @@ -2124,13 +2106,13 @@ async fn test_coalesce_propagate() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2170,7 +2152,7 @@ async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { // SortExec: expr=[b] <-- can't push this down // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order // SortExec: expr=[a] - // MemoryExec + // DataSourceExec let schema = create_test_schema3()?; let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); let sort_exprs_b = LexOrdering::new(vec![sort_expr("b", &schema)]); @@ -2186,7 +2168,7 @@ async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; // should not be able to push shorts let expected_no_change = expected_input; @@ -2200,7 +2182,7 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { // SortExec: expr=[a,b] <-- can push this down (as it is compatible with the required input ordering) // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order // SortExec: expr=[a] - // MemoryExec + // DataSourceExec let schema = create_test_schema3()?; let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); let sort_exprs_ab = @@ -2217,13 +2199,13 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; // should able to push shorts let expected = [ "RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected, plan, true); Ok(()) diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 565cee47e3b9..6e5c677541c5 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -57,7 +57,7 @@ async fn test_partial_final() -> Result<()> { let source = mock_data()?; let schema = source.schema(); - // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Partial/Final AggregateExec + // `SELECT a FROM DataSourceExec GROUP BY a LIMIT 4;`, Partial/Final AggregateExec let partial_agg = AggregateExec::try_new( AggregateMode::Partial, build_group_by(&schema.clone(), vec!["a".to_string()]), @@ -83,7 +83,7 @@ async fn test_partial_final() -> Result<()> { "LocalLimitExec: fetch=4", "AggregateExec: mode=Final, gby=[a@0 as a], aggr=[], lim=[4]", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], lim=[4]", - "MemoryExec: partitions=1, partition_sizes=[1]", + "DataSourceExec: partitions=1, partition_sizes=[1]", ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; @@ -107,7 +107,7 @@ async fn test_single_local() -> Result<()> { let source = mock_data()?; let schema = source.schema(); - // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Single AggregateExec + // `SELECT a FROM DataSourceExec GROUP BY a LIMIT 4;`, Single AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, build_group_by(&schema.clone(), vec!["a".to_string()]), @@ -124,7 +124,7 @@ async fn test_single_local() -> Result<()> { let expected = [ "LocalLimitExec: fetch=4", "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", - "MemoryExec: partitions=1, partition_sizes=[1]", + "DataSourceExec: partitions=1, partition_sizes=[1]", ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; @@ -148,7 +148,7 @@ async fn test_single_global() -> Result<()> { let source = mock_data()?; let schema = source.schema(); - // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Single AggregateExec + // `SELECT a FROM DataSourceExec GROUP BY a LIMIT 4;`, Single AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, build_group_by(&schema.clone(), vec!["a".to_string()]), @@ -166,7 +166,7 @@ async fn test_single_global() -> Result<()> { let expected = [ "GlobalLimitExec: skip=1, fetch=3", "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", - "MemoryExec: partitions=1, partition_sizes=[1]", + "DataSourceExec: partitions=1, partition_sizes=[1]", ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; @@ -189,7 +189,7 @@ async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { let source = mock_data()?; let schema = source.schema(); - // `SELECT distinct a FROM MemoryExec GROUP BY a, b LIMIT 4;`, Single/Single AggregateExec + // `SELECT distinct a FROM DataSourceExec GROUP BY a, b LIMIT 4;`, Single/Single AggregateExec let group_by_agg = AggregateExec::try_new( AggregateMode::Single, build_group_by(&schema.clone(), vec!["a".to_string(), "b".to_string()]), @@ -215,7 +215,7 @@ async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { "LocalLimitExec: fetch=4", "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[]", - "MemoryExec: partitions=1, partition_sizes=[1]", + "DataSourceExec: partitions=1, partition_sizes=[1]", ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; @@ -243,7 +243,7 @@ fn test_has_order_by() -> Result<()> { let source = parquet_exec_with_sort(vec![sort_key]); let schema = source.schema(); - // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec + // `SELECT a FROM DataSourceExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec // the `a > 1` filter is applied in the AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, @@ -261,7 +261,7 @@ fn test_has_order_by() -> Result<()> { let expected = [ "LocalLimitExec: fetch=10", "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], ordering_mode=Sorted", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; @@ -273,7 +273,7 @@ fn test_no_group_by() -> Result<()> { let source = mock_data()?; let schema = source.schema(); - // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec + // `SELECT FROM DataSourceExec LIMIT 10;`, Single AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, build_group_by(&schema, vec![]), @@ -290,7 +290,7 @@ fn test_no_group_by() -> Result<()> { let expected = [ "LocalLimitExec: fetch=10", "AggregateExec: mode=Single, gby=[], aggr=[]", - "MemoryExec: partitions=1, partition_sizes=[1]", + "DataSourceExec: partitions=1, partition_sizes=[1]", ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; @@ -303,7 +303,7 @@ fn test_has_aggregate_expression() -> Result<()> { let schema = source.schema(); let agg = TestAggregate::new_count_star(); - // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec + // `SELECT FROM DataSourceExec LIMIT 10;`, Single AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, build_group_by(&schema, vec!["a".to_string()]), @@ -320,7 +320,7 @@ fn test_has_aggregate_expression() -> Result<()> { let expected = [ "LocalLimitExec: fetch=10", "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[COUNT(*)]", - "MemoryExec: partitions=1, partition_sizes=[1]", + "DataSourceExec: partitions=1, partition_sizes=[1]", ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; @@ -332,7 +332,7 @@ fn test_has_filter() -> Result<()> { let source = mock_data()?; let schema = source.schema(); - // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec + // `SELECT a FROM DataSourceExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec // the `a > 1` filter is applied in the AggregateExec let filter_expr = Some(expressions::binary( col("a", &schema)?, @@ -358,7 +358,7 @@ fn test_has_filter() -> Result<()> { let expected = [ "LocalLimitExec: fetch=10", "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[COUNT(*)]", - "MemoryExec: partitions=1, partition_sizes=[1]", + "DataSourceExec: partitions=1, partition_sizes=[1]", ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index fc576e929591..12b41196ef37 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -19,9 +19,8 @@ use std::any::Any; use std::sync::Arc; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; -use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig}; +use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig}; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_common::{JoinSide, JoinType, ScalarValue}; @@ -48,11 +47,12 @@ use datafusion_physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; -use datafusion_physical_plan::memory::MemoryExec; +use datafusion_physical_plan::memory::MemorySourceConfig; use datafusion_physical_plan::projection::{update_expr, ProjectionExec}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::streaming::PartitionStream; use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; @@ -374,21 +374,14 @@ fn create_simple_csv_exec() -> Arc { Field::new("d", DataType::Int32, true), Field::new("e", DataType::Int32, true), ])); - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![0, 1, 2, 3, 4])), - ) - .with_has_header(false) - .with_delimeter(0) - .with_quote(0) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema, + Arc::new(CsvSource::new(false, 0, 0)), ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![0, 1, 2, 3, 4])) + .new_exec() } fn create_projecting_csv_exec() -> Arc { @@ -398,21 +391,14 @@ fn create_projecting_csv_exec() -> Arc { Field::new("c", DataType::Int32, true), Field::new("d", DataType::Int32, true), ])); - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![3, 2, 1])), - ) - .with_has_header(false) - .with_delimeter(0) - .with_quote(0) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema, + Arc::new(CsvSource::new(false, 0, 0)), ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![3, 2, 1])) + .new_exec() } fn create_projecting_memory_exec() -> Arc { @@ -424,7 +410,7 @@ fn create_projecting_memory_exec() -> Arc { Field::new("e", DataType::Int32, true), ])); - Arc::new(MemoryExec::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap()) + MemorySourceConfig::try_new_exec(&[], schema, Some(vec![2, 0, 3, 4])).unwrap() } #[test] @@ -440,7 +426,7 @@ fn test_csv_after_projection() -> Result<()> { let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[b@2 as b, d@0 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[d, c, b], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[d, c, b], file_type=csv, has_header=false", ]; assert_eq!(initial, expected_initial); @@ -448,7 +434,7 @@ fn test_csv_after_projection() -> Result<()> { ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; let expected = - ["CsvExec: file_groups={1 group: [[x]]}, projection=[b, d], has_header=false"]; + ["DataSourceExec: file_groups={1 group: [[x]]}, projection=[b, d], file_type=csv, has_header=false"]; assert_eq!(get_plan_string(&after_optimize), expected); Ok(()) @@ -468,20 +454,24 @@ fn test_memory_after_projection() -> Result<()> { let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[d@2 as d, e@3 as e, a@1 as a]", - " MemoryExec: partitions=0, partition_sizes=[]", + " DataSourceExec: partitions=0, partition_sizes=[]", ]; assert_eq!(initial, expected_initial); let after_optimize = ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - let expected = ["MemoryExec: partitions=0, partition_sizes=[]"]; + let expected = ["DataSourceExec: partitions=0, partition_sizes=[]"]; assert_eq!(get_plan_string(&after_optimize), expected); assert_eq!( after_optimize .clone() .as_any() - .downcast_ref::() + .downcast_ref::() + .unwrap() + .source() + .as_any() + .downcast_ref::() .unwrap() .projection() .clone() @@ -639,7 +629,7 @@ fn test_projection_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[new_b@3 as new_b, c@0 + new_e@1 as binary, new_b@3 as newest_b]", " ProjectionExec: expr=[c@2 as c, e@4 as new_e, a@0 as a, b@1 as new_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(initial, expected_initial); @@ -648,7 +638,7 @@ fn test_projection_after_projection() -> Result<()> { let expected = [ "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -692,7 +682,7 @@ fn test_output_req_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", " OutputRequirementExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(initial, expected_initial); @@ -702,7 +692,7 @@ fn test_output_req_after_projection() -> Result<()> { let expected: [&str; 3] = [ "OutputRequirementExec", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -769,7 +759,7 @@ fn test_coalesce_partitions_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", " CoalescePartitionsExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_eq!(initial, expected_initial); @@ -779,7 +769,7 @@ fn test_coalesce_partitions_after_projection() -> Result<()> { let expected = [ "CoalescePartitionsExec", " ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -816,7 +806,7 @@ fn test_filter_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", " FilterExec: b@1 - a@0 > d@3 - a@0", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_eq!(initial, expected_initial); @@ -826,7 +816,7 @@ fn test_filter_after_projection() -> Result<()> { let expected = [ "FilterExec: b@1 - a_new@0 > d@2 - a_new@0", " ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -901,8 +891,8 @@ fn test_join_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(initial, expected_initial); @@ -912,9 +902,9 @@ fn test_join_after_projection() -> Result<()> { let expected = [ "SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b_from_left@1, c_from_right@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", " ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " ProjectionExec: expr=[a@0 as a_from_right, c@2 as c_from_right]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -1020,8 +1010,8 @@ fn test_join_after_required_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(initial, expected_initial); @@ -1031,8 +1021,8 @@ fn test_join_after_required_projection() -> Result<()> { let expected = [ "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); Ok(()) @@ -1088,8 +1078,8 @@ fn test_nested_loop_join_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[c@2 as c]", " NestedLoopJoinExec: join_type=Inner, filter=a@0 < b@1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_eq!(initial, expected_initial); @@ -1097,8 +1087,8 @@ fn test_nested_loop_join_after_projection() -> Result<()> { ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; let expected = [ "NestedLoopJoinExec: join_type=Inner, filter=a@0 < b@1, projection=[c@2]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_eq!(get_plan_string(&after_optimize), expected); Ok(()) @@ -1170,7 +1160,7 @@ fn test_hash_join_after_projection() -> Result<()> { )?); let initial = get_plan_string(&projection); let expected_initial = [ - "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@7 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@7 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(initial, expected_initial); @@ -1178,7 +1168,7 @@ fn test_hash_join_after_projection() -> Result<()> { ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; // HashJoinExec only returns result after projection. Because there are some alias columns in the projection, the ProjectionExec is not removed. - let expected = ["ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@3 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + let expected = ["ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@3 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"]; assert_eq!(get_plan_string(&after_optimize), expected); let projection: Arc = Arc::new(ProjectionExec::try_new( @@ -1195,7 +1185,7 @@ fn test_hash_join_after_projection() -> Result<()> { ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; // Comparing to the previous result, this projection don't have alias columns either change the order of output fields. So the ProjectionExec is removed. - let expected = ["HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + let expected = ["HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"]; assert_eq!(get_plan_string(&after_optimize), expected); Ok(()) @@ -1227,7 +1217,7 @@ fn test_repartition_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", " RepartitionExec: partitioning=Hash([a@0, b@1, d@3], 6), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_eq!(initial, expected_initial); @@ -1237,7 +1227,7 @@ fn test_repartition_after_projection() -> Result<()> { let expected = [ "RepartitionExec: partitioning=Hash([a@1, b_new@0, d_new@2], 6), input_partitions=1", " ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -1294,7 +1284,7 @@ fn test_sort_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", " SortExec: expr=[b@1 ASC, c@2 + a@0 ASC], preserve_partitioning=[false]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(initial, expected_initial); @@ -1304,7 +1294,7 @@ fn test_sort_after_projection() -> Result<()> { let expected = [ "SortExec: expr=[b@2 ASC, c@0 + new_a@1 ASC], preserve_partitioning=[false]", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -1344,7 +1334,7 @@ fn test_sort_preserving_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", " SortPreservingMergeExec: [b@1 ASC, c@2 + a@0 ASC]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(initial, expected_initial); @@ -1354,7 +1344,7 @@ fn test_sort_preserving_after_projection() -> Result<()> { let expected = [ "SortPreservingMergeExec: [b@2 ASC, c@0 + new_a@1 ASC]", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -1379,9 +1369,9 @@ fn test_union_after_projection() -> Result<()> { let expected_initial = [ "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", " UnionExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(initial, expected_initial); @@ -1391,11 +1381,11 @@ fn test_union_after_projection() -> Result<()> { let expected = [ "UnionExec", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 52cd5e5754fa..9a68d38f96c6 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -32,13 +32,14 @@ use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::collect; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; -use datafusion_physical_plan::memory::MemoryExec; +use datafusion_physical_plan::memory::MemorySourceConfig; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ displayable, get_plan_string, ExecutionPlan, Partitioning, }; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::Result; use datafusion_expr::{JoinType, Operator}; @@ -212,7 +213,7 @@ async fn test_replace_multiple_input_repartition_1( " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -229,13 +230,13 @@ async fn test_replace_multiple_input_repartition_1( " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -302,7 +303,7 @@ async fn test_with_inter_children_change_only( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", ]; // Expected unbounded result (same for with and without flag) @@ -328,7 +329,7 @@ async fn test_with_inter_children_change_only( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC]", @@ -338,7 +339,7 @@ async fn test_with_inter_children_change_only( " SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -388,7 +389,7 @@ async fn test_replace_multiple_input_repartition_2( " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -407,14 +408,14 @@ async fn test_replace_multiple_input_repartition_2( " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -467,7 +468,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -488,7 +489,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -496,7 +497,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -552,7 +553,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -575,7 +576,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -584,7 +585,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -635,7 +636,7 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -655,7 +656,7 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -713,7 +714,7 @@ async fn test_with_multiple_replacable_repartitions( " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -736,7 +737,7 @@ async fn test_with_multiple_replacable_repartitions( " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -745,7 +746,7 @@ async fn test_with_multiple_replacable_repartitions( " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -797,7 +798,7 @@ async fn test_not_replace_with_different_orderings( " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -815,7 +816,7 @@ async fn test_not_replace_with_different_orderings( " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -864,7 +865,7 @@ async fn test_with_lost_ordering( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -881,13 +882,13 @@ async fn test_with_lost_ordering( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -954,7 +955,7 @@ async fn test_with_lost_and_kept_ordering( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -981,7 +982,7 @@ async fn test_with_lost_and_kept_ordering( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [c@1 ASC]", @@ -992,7 +993,7 @@ async fn test_with_lost_and_kept_ordering( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -1069,11 +1070,11 @@ async fn test_with_multiple_child_trees( " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; // Expected unbounded result (same for with and without flag) @@ -1100,11 +1101,11 @@ async fn test_with_multiple_child_trees( " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -1259,9 +1260,11 @@ fn memory_exec_sorted( .map(|_| vec![make_partition(schema, rows)]) .collect(); let projection: Vec = vec![0, 2, 3]; - MemoryExec::try_new(&data, schema.clone(), Some(projection)) - .unwrap() - .try_with_sort_information(vec![sort_exprs]) - .unwrap() + DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&data, schema.clone(), Some(projection)) + .unwrap() + .try_with_sort_information(vec![sort_exprs]) + .unwrap(), + )) }) } diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index 3057ca819e82..ccfec1fcb10e 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -422,7 +422,7 @@ async fn test_bounded_window_agg_sort_requirement() -> Result<()> { assert_plan(bw.as_ref(), vec![ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]" + " DataSourceExec: partitions=1, partition_sizes=[0]" ]); assert_sanity_check(&bw, true); Ok(()) @@ -444,7 +444,7 @@ async fn test_bounded_window_agg_no_sort_requirement() -> Result<()> { let bw = bounded_window_exec("c9", sort_exprs, source); assert_plan(bw.as_ref(), vec![ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " MemoryExec: partitions=1, partition_sizes=[0]" + " DataSourceExec: partitions=1, partition_sizes=[0]" ]); // Order requirement of the `BoundedWindowAggExec` is not satisfied. We expect to receive error during sanity check. assert_sanity_check(&bw, false); @@ -463,7 +463,7 @@ async fn test_global_limit_single_partition() -> Result<()> { limit.as_ref(), vec![ "GlobalLimitExec: skip=0, fetch=100", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ], ); assert_sanity_check(&limit, true); @@ -483,7 +483,7 @@ async fn test_global_limit_multi_partition() -> Result<()> { vec![ "GlobalLimitExec: skip=0, fetch=100", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ], ); // Distribution requirement of the `GlobalLimitExec` is not satisfied. We expect to receive error during sanity check. @@ -502,7 +502,7 @@ async fn test_local_limit() -> Result<()> { limit.as_ref(), vec![ "LocalLimitExec: fetch=100", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ], ); assert_sanity_check(&limit, true); @@ -543,10 +543,10 @@ async fn test_sort_merge_join_satisfied() -> Result<()> { "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ], ); assert_sanity_check(&smj, true); @@ -590,9 +590,9 @@ async fn test_sort_merge_join_order_missing() -> Result<()> { "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ], ); // Order requirement for the `SortMergeJoin` is not satisfied for right child. We expect to receive error during sanity check. @@ -636,10 +636,10 @@ async fn test_sort_merge_join_dist_missing() -> Result<()> { "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ], ); // Distribution requirement for the `SortMergeJoin` is not satisfied for right child (has round-robin partitioning). We expect to receive error during sanity check. diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index b26b1227ea43..fb36be3da1b4 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -25,7 +25,7 @@ use arrow::array::Int32Array; use arrow::record_batch::RecordBatch; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; @@ -51,10 +51,11 @@ use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{JoinFilter, JoinOn}; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion_physical_plan::memory::MemoryExec; +use datafusion_physical_plan::memory::MemorySourceConfig; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::union::UnionExec; @@ -66,24 +67,28 @@ use datafusion_physical_plan::{ use datafusion_physical_plan::{InputOrderMode, Partitioning}; /// Create a non sorted parquet exec -pub fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)), +pub fn parquet_exec(schema: &SchemaRef) -> Arc { + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + Arc::new(ParquetSource::default()), ) - .build_arc() + .with_file(PartitionedFile::new("x".to_string(), 100)) + .new_exec() } /// Create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( output_ordering: Vec, -) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), +) -> Arc { + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(ParquetSource::default()), ) - .build_arc() + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering) + .new_exec() } pub fn schema() -> SchemaRef { @@ -174,7 +179,7 @@ pub fn coalesce_partitions_exec(input: Arc) -> Arc Arc { - Arc::new(MemoryExec::try_new(&[vec![]], Arc::clone(schema), None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() } pub fn hash_join_exec( @@ -521,7 +526,7 @@ pub fn stream_exec_ordered_with_projection( ) } -pub fn mock_data() -> Result> { +pub fn mock_data() -> Result> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), @@ -549,11 +554,7 @@ pub fn mock_data() -> Result> { ], )?; - Ok(Arc::new(MemoryExec::try_new( - &[vec![batch]], - Arc::clone(&schema), - None, - )?)) + MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) } pub fn build_group_by(input_schema: &SchemaRef, columns: Vec) -> PhysicalGroupBy { diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index dce175d04b2b..d4b5ae8b2820 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -610,7 +610,7 @@ async fn test_physical_plan_display_indent() { " CoalesceBatchesExec: target_batch_size=4096", " FilterExec: c12@1 < 10", " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], has_header=true", + " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], file_type=csv, has_header=true", ]; let normalizer = ExplainNormalizer::new(); @@ -650,12 +650,12 @@ async fn test_physical_plan_display_indent_multi_children() { " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", + " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=9000", " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", " ProjectionExec: expr=[c1@0 as c2]", - " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", + " DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true", ]; let normalizer = ExplainNormalizer::new(); diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index fab92c0f9c2b..77eec20eac00 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -67,11 +67,11 @@ async fn join_change_in_planner() -> Result<()> { " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a1@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false", + // " DataSourceExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], file_type=csv, has_header=false", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a1@0 ASC NULLS LAST", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" + // " DataSourceExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], file_type=csv, has_header=false" ] }; let mut actual: Vec<&str> = formatted.trim().lines().collect(); @@ -135,11 +135,11 @@ async fn join_no_order_on_filter() -> Result<()> { " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false", + // " DataSourceExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], file_type=csv, has_header=false", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" + // " DataSourceExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], file_type=csv, has_header=false" ] }; let mut actual: Vec<&str> = formatted.trim().lines().collect(); @@ -185,11 +185,11 @@ async fn join_change_in_planner_without_sort() -> Result<()> { " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false", + // " DataSourceExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], file_type=csv, has_header=false", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" + // " DataSourceExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], file_type=csv, has_header=false" ] }; let mut actual: Vec<&str> = formatted.trim().lines().collect(); diff --git a/datafusion/core/tests/sql/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs index c4fa4c509aa8..6345f5e4352f 100644 --- a/datafusion/core/tests/sql/path_partition.rs +++ b/datafusion/core/tests/sql/path_partition.rs @@ -25,7 +25,7 @@ use std::sync::Arc; use arrow::datatypes::DataType; use datafusion::datasource::listing::ListingTableUrl; -use datafusion::datasource::physical_plan::ParquetExec; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::{ assert_batches_sorted_eq, datasource::{ @@ -41,12 +41,13 @@ use datafusion_catalog::TableProvider; use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; use datafusion_execution::config::SessionConfig; +use datafusion_expr::{col, lit, Expr, Operator}; +use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; +use datafusion_physical_plan::source::DataSourceExec; use async_trait::async_trait; use bytes::Bytes; use chrono::{TimeZone, Utc}; -use datafusion_expr::{col, lit, Expr, Operator}; -use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; use futures::stream::{self, BoxStream}; use object_store::{ path::Path, GetOptions, GetResult, GetResultPayload, ListResult, ObjectMeta, @@ -84,8 +85,15 @@ async fn parquet_partition_pruning_filter() -> Result<()> { Expr::gt(col("id"), lit(1)), ]; let exec = table.scan(&ctx.state(), None, &filters, None).await?; - let parquet_exec = exec.as_any().downcast_ref::().unwrap(); - let pred = parquet_exec.predicate().unwrap(); + let data_source = exec.as_any().downcast_ref::().unwrap(); + let source = data_source.source(); + let file_source = source.as_any().downcast_ref::().unwrap(); + let parquet_config = file_source + .file_source() + .as_any() + .downcast_ref::() + .unwrap(); + let pred = parquet_config.predicate().unwrap(); // Only the last filter should be pushdown to TableScan let expected = Arc::new(BinaryExpr::new( Arc::new(Column::new_with_schema("id", &exec.schema()).unwrap()), diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs index b5f94107dd0b..0ec9a5fd7620 100644 --- a/datafusion/core/tests/user_defined/user_defined_table_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -15,26 +15,29 @@ // specific language governing permissions and limitations // under the License. +use std::fs::File; +use std::io::Seek; +use std::path::Path; +use std::sync::Arc; + use arrow::array::Int64Array; use arrow::csv::reader::Format; use arrow::csv::ReaderBuilder; -use async_trait::async_trait; + use datafusion::arrow::datatypes::SchemaRef; use datafusion::arrow::record_batch::RecordBatch; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::execution::TaskContext; -use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::SessionContext; use datafusion_catalog::Session; use datafusion_catalog::TableFunctionImpl; use datafusion_common::{assert_batches_eq, DFSchema, ScalarValue}; use datafusion_expr::{EmptyRelation, Expr, LogicalPlan, Projection, TableType}; -use std::fs::File; -use std::io::Seek; -use std::path::Path; -use std::sync::Arc; +use datafusion_physical_plan::memory::MemorySourceConfig; + +use async_trait::async_trait; /// test simple udtf with define read_csv with parameters #[tokio::test] @@ -153,11 +156,11 @@ impl TableProvider for SimpleCsvTable { } else { self.batches.clone() }; - Ok(Arc::new(MemoryExec::try_new( + Ok(MemorySourceConfig::try_new_exec( &[batches], TableProvider::schema(self), projection.cloned(), - )?)) + )?) } } diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index 797f2687aba6..5e49debac7e7 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -54,7 +54,7 @@ pub use pool::*; /// As explained above, DataFusion's design ONLY limits operators that require /// "large" amounts of memory (proportional to number of input rows), such as /// `GroupByHashExec`. It does NOT track and limit memory used internally by -/// other operators such as `ParquetExec` or the `RecordBatch`es that flow +/// other operators such as `DataSourceExec` or the `RecordBatch`es that flow /// between operators. /// /// In order to avoid allocating memory until the OS or the container system diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 176c1e69aba8..d0f7b3671e60 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -975,14 +975,14 @@ fn add_spm_on_top(input: DistributionContext) -> DistributionContext { /// ```text /// "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", -/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// " DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=parquet", /// ``` /// /// Since `RepartitionExec`s change the distribution, this function removes /// them and returns following plan: /// /// ```text -/// "ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// "DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=parquet", /// ``` fn remove_dist_changing_operators( mut distribution_context: DistributionContext, @@ -1007,7 +1007,7 @@ fn remove_dist_changing_operators( /// "SortPreservingMergeExec: \[a@0 ASC]" /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true", /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true", -/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// " DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=parquet", /// ``` /// /// This function converts plan above to the following: @@ -1016,7 +1016,7 @@ fn remove_dist_changing_operators( /// "CoalescePartitionsExec" /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", -/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// " DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=parquet", /// ``` fn replace_order_preserving_variants( mut context: DistributionContext, diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index fbbd27409173..08feb9bbe04f 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -23,12 +23,12 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::memory::MemoryExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{collect, ExecutionPlan}; use criterion::async_executor::FuturesExecutor; use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_physical_plan::memory::MemorySourceConfig; fn generate_spm_for_round_robin_tie_breaker( has_same_value: bool, @@ -82,8 +82,8 @@ fn generate_spm_for_round_robin_tie_breaker( }, ]); - let exec = MemoryExec::try_new(&partitiones, schema, None).unwrap(); - SortPreservingMergeExec::new(sort, Arc::new(exec)) + let exec = MemorySourceConfig::try_new_exec(&partitiones, schema, None).unwrap(); + SortPreservingMergeExec::new(sort, exec) .with_round_robin_repartition(enable_round_robin_repartition) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3aaa766f92f2..b4f54b0d943a 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1345,9 +1345,10 @@ mod tests { use crate::coalesce_batches::CoalesceBatchesExec; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::common; + use crate::common::collect; use crate::execution_plan::Boundedness; use crate::expressions::col; - use crate::memory::MemoryExec; + use crate::memory::MemorySourceConfig; use crate::metrics::MetricValue; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; @@ -1372,13 +1373,12 @@ mod tests { use datafusion_functions_aggregate::first_last::{first_value_udaf, last_value_udaf}; use datafusion_functions_aggregate::median::median_udaf; use datafusion_functions_aggregate::sum::sum_udaf; - use datafusion_physical_expr::expressions::lit; - use datafusion_physical_expr::PhysicalSortExpr; - - use crate::common::collect; use datafusion_physical_expr::aggregate::AggregateExprBuilder; + use datafusion_physical_expr::expressions::lit; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::Partitioning; + use datafusion_physical_expr::PhysicalSortExpr; + use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -2166,14 +2166,14 @@ mod tests { // " CoalesceBatchesExec: target_batch_size=1024", // " CoalescePartitionsExec", // " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[FIRST_VALUE(b)], ordering_mode=None", - // " MemoryExec: partitions=4, partition_sizes=[1, 1, 1, 1]", + // " DataSourceExec: partitions=4, partition_sizes=[1, 1, 1, 1]", // // or // // "AggregateExec: mode=Final, gby=[a@0 as a], aggr=[FIRST_VALUE(b)]", // " CoalescePartitionsExec", // " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[FIRST_VALUE(b)], ordering_mode=None", - // " MemoryExec: partitions=4, partition_sizes=[1, 1, 1, 1]", + // " DataSourceExec: partitions=4, partition_sizes=[1, 1, 1, 1]", // // and checks whether the function `merge_batch` works correctly for // FIRST_VALUE and LAST_VALUE functions. @@ -2208,7 +2208,7 @@ mod tests { vec![test_last_value_agg_expr(&schema, sort_options)?] }; - let memory_exec = Arc::new(MemoryExec::try_new( + let memory_exec = MemorySourceConfig::try_new_exec( &[ vec![partition1], vec![partition2], @@ -2217,7 +2217,7 @@ mod tests { ], Arc::clone(&schema), None, - )?); + )?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups.clone(), @@ -2443,11 +2443,11 @@ mod tests { }) .collect(); - let input = Arc::new(MemoryExec::try_new( + let input = MemorySourceConfig::try_new_exec( &[input_batches], Arc::clone(&schema), None, - )?); + )?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Single, @@ -2558,11 +2558,11 @@ mod tests { .build() .map(Arc::new)?]; - let input = Arc::new(MemoryExec::try_new( + let input = MemorySourceConfig::try_new_exec( &[vec![batch.clone()]], Arc::::clone(&batch.schema()), None, - )?); + )?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::FinalPartitioned, group_by, @@ -2627,11 +2627,8 @@ mod tests { .unwrap(), ]; - let input = Arc::new(MemoryExec::try_new( - &[input_data], - Arc::clone(&schema), - None, - )?); + let input = + MemorySourceConfig::try_new_exec(&[input_data], Arc::clone(&schema), None)?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, @@ -2717,11 +2714,8 @@ mod tests { .unwrap(), ]; - let input = Arc::new(MemoryExec::try_new( - &[input_data], - Arc::clone(&schema), - None, - )?); + let input = + MemorySourceConfig::try_new_exec(&[input_data], Arc::clone(&schema), None)?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, @@ -2836,7 +2830,7 @@ mod tests { create_record_batch(&schema, (vec![2, 3, 4, 4], vec![1.0, 2.0, 3.0, 4.0]))?, ]; let plan: Arc = - Arc::new(MemoryExec::try_new(&[batches], Arc::clone(&schema), None)?); + MemorySourceConfig::try_new_exec(&[batches], Arc::clone(&schema), None)?; let grouping_set = PhysicalGroupBy::new( vec![(col("a", &schema)?, "a".to_string())], diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 961d2f639897..dbf82eee05eb 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -141,7 +141,7 @@ impl<'a> DisplayableExecutionPlan<'a> { /// CoalesceBatchesExec: target_batch_size=8192 /// FilterExec: a < 5 /// RepartitionExec: partitioning=RoundRobinBatch(16) - /// CsvExec: source=...", + /// DataSourceExec: source=...", /// ``` pub fn indent(&self, verbose: bool) -> impl fmt::Display + 'a { let format_type = if verbose { diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index a54b46111f53..da3d33cd0e21 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -74,7 +74,7 @@ use tokio::task::JoinSet; /// [`required_input_distribution`]: ExecutionPlan::required_input_distribution /// [`required_input_ordering`]: ExecutionPlan::required_input_ordering pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { - /// Short name for the ExecutionPlan, such as 'ParquetExec'. + /// Short name for the ExecutionPlan, such as 'DataSourceExec'. /// /// Implementation note: this method can just proxy to /// [`static_name`](ExecutionPlan::static_name) if no special action is @@ -83,7 +83,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// range of use cases. fn name(&self) -> &str; - /// Short name for the ExecutionPlan, such as 'ParquetExec'. + /// Short name for the ExecutionPlan, such as 'DataSourceExec'. /// Like [`name`](ExecutionPlan::name) but can be called without an instance. fn static_name() -> &'static str where @@ -600,10 +600,10 @@ impl Boundedness { /// |_ on: [col1 ASC] /// FilterExec [EmissionType::Incremental] /// |_ pred: col2 > 100 -/// CsvExec [EmissionType::Incremental] +/// DataSourceExec [EmissionType::Incremental] /// |_ file: "data.csv" /// ``` -/// - CsvExec emits records incrementally as it reads from the file +/// - DataSourceExec emits records incrementally as it reads from the file /// - FilterExec processes and emits filtered records incrementally as they arrive /// - SortExec must wait for all input records before it can emit the sorted result, /// since it needs to see all values to determine their final order @@ -778,7 +778,7 @@ impl PlanProperties { } /// Get schema of the node. - fn schema(&self) -> &SchemaRef { + pub(crate) fn schema(&self) -> &SchemaRef { self.eq_properties.schema() } } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 6d33a3a56eac..76e535d93b7e 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1638,9 +1638,10 @@ impl EmbeddedProjection for HashJoinExec { #[cfg(test)] mod tests { use super::*; + use crate::memory::MemorySourceConfig; use crate::{ - common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, - test::build_table_i32, test::exec::MockExec, + common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, + test::exec::MockExec, }; use arrow::array::{Date32Array, Int32Array}; @@ -1656,7 +1657,6 @@ mod tests { use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::PhysicalExpr; - use hashbrown::raw::RawTable; use rstest::*; use rstest_reuse::*; @@ -1681,7 +1681,7 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn join( @@ -2083,9 +2083,9 @@ mod tests { let batch2 = build_table_i32(("a1", &vec![2]), ("b2", &vec![2]), ("c1", &vec![9])); let schema = batch1.schema(); - let left = Arc::new( - MemoryExec::try_new(&[vec![batch1], vec![batch2]], schema, None).unwrap(), - ); + let left = + MemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + .unwrap(); let right = build_table( ("a1", &vec![1, 2, 3]), @@ -2155,9 +2155,9 @@ mod tests { ); let schema = batch1.schema(); - let left = Arc::new( - MemoryExec::try_new(&[vec![batch1], vec![batch2]], schema, None).unwrap(), - ); + let left = + MemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + .unwrap(); let right = build_table( ("a2", &vec![20, 30, 10]), ("b2", &vec![5, 6, 4]), @@ -2209,9 +2209,9 @@ mod tests { let batch2 = build_table_i32(("a2", &vec![30]), ("b1", &vec![5]), ("c2", &vec![90])); let schema = batch1.schema(); - let right = Arc::new( - MemoryExec::try_new(&[vec![batch1], vec![batch2]], schema, None).unwrap(), - ); + let right = + MemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + .unwrap(); let on = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?) as _, @@ -2289,9 +2289,8 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - Arc::new( - MemoryExec::try_new(&[vec![batch.clone(), batch]], schema, None).unwrap(), - ) + MemorySourceConfig::try_new_exec(&[vec![batch.clone(), batch]], schema, None) + .unwrap() } #[apply(batch_sizes)] @@ -2396,7 +2395,8 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, )]; let schema = right.schema(); - let right = Arc::new(MemoryExec::try_new(&[vec![right]], schema, None).unwrap()); + let right = + MemorySourceConfig::try_new_exec(&[vec![right]], schema, None).unwrap(); let join = join(left, right, on, &JoinType::Left, false).unwrap(); let columns = columns(&join.schema()); @@ -2433,7 +2433,8 @@ mod tests { Arc::new(Column::new_with_schema("b2", &right.schema()).unwrap()) as _, )]; let schema = right.schema(); - let right = Arc::new(MemoryExec::try_new(&[vec![right]], schema, None).unwrap()); + let right = + MemorySourceConfig::try_new_exec(&[vec![right]], schema, None).unwrap(); let join = join(left, right, on, &JoinType::Full, false).unwrap(); let columns = columns(&join.schema()); @@ -3737,15 +3738,14 @@ mod tests { let dates: ArrayRef = Arc::new(Date32Array::from(vec![19107, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; - let left = Arc::new( - MemoryExec::try_new(&[vec![batch]], Arc::clone(&schema), None).unwrap(), - ); - + let left = + MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) + .unwrap(); let dates: ArrayRef = Arc::new(Date32Array::from(vec![19108, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; - let right = Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()); - + let right = + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); let on = vec![( Arc::new(Column::new_with_schema("date", &left.schema()).unwrap()) as _, Arc::new(Column::new_with_schema("date", &right.schema()).unwrap()) as _, @@ -4035,27 +4035,23 @@ mod tests { ("b1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), ("c1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), ); - let left = Arc::new( - MemoryExec::try_new( - &[vec![left_batch.clone()], vec![left_batch.clone()]], - left_batch.schema(), - None, - ) - .unwrap(), - ); + let left = MemorySourceConfig::try_new_exec( + &[vec![left_batch.clone()], vec![left_batch.clone()]], + left_batch.schema(), + None, + ) + .unwrap(); let right_batch = build_table_i32( ("a2", &vec![10, 11]), ("b2", &vec![12, 13]), ("c2", &vec![14, 15]), ); - let right = Arc::new( - MemoryExec::try_new( - &[vec![right_batch.clone()], vec![right_batch.clone()]], - right_batch.schema(), - None, - ) - .unwrap(), - ); + let right = MemorySourceConfig::try_new_exec( + &[vec![right_batch.clone()], vec![right_batch.clone()]], + right_batch.schema(), + None, + ) + .unwrap(); let on = vec![( Arc::new(Column::new_with_schema("b1", &left_batch.schema())?) as _, Arc::new(Column::new_with_schema("b2", &right_batch.schema())?) as _, @@ -4135,7 +4131,7 @@ mod tests { ) .unwrap(); let schema_ref = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema_ref, None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![batch]], schema_ref, None).unwrap() } #[tokio::test] diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index ce960df32ec2..50c411ccd785 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1030,9 +1030,10 @@ impl EmbeddedProjection for NestedLoopJoinExec { #[cfg(test)] pub(crate) mod tests { use super::*; + use crate::memory::MemorySourceConfig; + use crate::source::DataSourceExec; use crate::{ - common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, - test::build_table_i32, + common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, }; use arrow::datatypes::{DataType, Field}; @@ -1070,8 +1071,8 @@ pub(crate) mod tests { vec![batch] }; - let mut exec = - MemoryExec::try_new(&[batches], Arc::clone(&schema), None).unwrap(); + let mut source = + MemorySourceConfig::try_new(&[batches], Arc::clone(&schema), None).unwrap(); if !sorted_column_names.is_empty() { let mut sort_info = LexOrdering::default(); for name in sorted_column_names { @@ -1085,10 +1086,10 @@ pub(crate) mod tests { }; sort_info.push(sort_expr); } - exec = exec.try_with_sort_information(vec![sort_info]).unwrap(); + source = source.try_with_sort_information(vec![sort_info]).unwrap(); } - Arc::new(exec) + Arc::new(DataSourceExec::new(Arc::new(source))) } fn build_left_table() -> Arc { diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 6565e6799b83..5570ceb12ae0 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -2548,7 +2548,7 @@ mod tests { use crate::joins::sort_merge_join::{get_corrected_filter_mask, JoinedRecordBatches}; use crate::joins::utils::{ColumnIndex, JoinFilter, JoinOn}; use crate::joins::SortMergeJoinExec; - use crate::memory::MemoryExec; + use crate::memory::MemorySourceConfig; use crate::test::{build_table_i32, build_table_i32_two_cols}; use crate::{common, ExecutionPlan}; @@ -2559,12 +2559,12 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn build_table_from_batches(batches: Vec) -> Arc { let schema = batches.first().unwrap().schema(); - Arc::new(MemoryExec::try_new(&[batches], schema, None).unwrap()) + MemorySourceConfig::try_new_exec(&[batches], schema, None).unwrap() } fn build_date_table( @@ -2589,7 +2589,7 @@ mod tests { .unwrap(); let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn build_date64_table( @@ -2614,7 +2614,7 @@ mod tests { .unwrap(); let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } /// returns a table with 3 columns of i32 in memory @@ -2637,7 +2637,7 @@ mod tests { ], ) .unwrap(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } pub fn build_table_two_cols( @@ -2646,7 +2646,7 @@ mod tests { ) -> Arc { let batch = build_table_i32_two_cols(a, b); let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn join( diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 0e3d03fcae94..a866f7291bbd 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -23,8 +23,9 @@ use crate::joins::utils::{JoinFilter, JoinOn}; use crate::joins::{ HashJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; -use crate::memory::MemoryExec; +use crate::memory::MemorySourceConfig; use crate::repartition::RepartitionExec; +use crate::source::DataSourceExec; use crate::{common, ExecutionPlan, ExecutionPlanProperties, Partitioning}; use arrow::array::types::IntervalDayTime; @@ -530,12 +531,15 @@ pub fn create_memory_table( right_sorted: Vec, ) -> Result<(Arc, Arc)> { let left_schema = left_partition[0].schema(); - let left = MemoryExec::try_new(&[left_partition], left_schema, None)? + let left = MemorySourceConfig::try_new(&[left_partition], left_schema, None)? .try_with_sort_information(left_sorted)?; let right_schema = right_partition[0].schema(); - let right = MemoryExec::try_new(&[right_partition], right_schema, None)? + let right = MemorySourceConfig::try_new(&[right_partition], right_schema, None)? .try_with_sort_information(right_sorted)?; - Ok((Arc::new(left), Arc::new(right))) + Ok(( + Arc::new(DataSourceExec::new(Arc::new(left))), + Arc::new(DataSourceExec::new(Arc::new(right))), + )) } /// Filter expr for a + b > c + 10 AND a + b < c + 100 diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 5ad37f0b1ac0..9210e3b0273c 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -86,5 +86,6 @@ pub mod udaf { } pub mod coalesce; +pub mod source; #[cfg(test)] pub mod test; diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 198b8ccd6992..5607ad9e2a9f 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -31,6 +31,7 @@ use crate::execution_plan::{Boundedness, EmissionType}; use crate::projection::{ all_alias_free_columns, new_projections_for_columns, ProjectionExec, }; +use crate::source::{DataSource, DataSourceExec}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -51,73 +52,37 @@ use parking_lot::RwLock; /// Execution plan for reading in-memory batches of data #[derive(Clone)] +#[deprecated( + since = "46.0.0", + note = "use MemorySourceConfig and DataSourceExec instead" +)] pub struct MemoryExec { + inner: DataSourceExec, /// The partitions to query partitions: Vec>, - /// Schema representing the data before projection - schema: SchemaRef, - /// Schema representing the data after the optional projection is applied - projected_schema: SchemaRef, /// Optional projection projection: Option>, // Sort information: one or more equivalent orderings sort_information: Vec, - cache: PlanProperties, /// if partition sizes should be displayed show_sizes: bool, } +#[allow(unused, deprecated)] impl fmt::Debug for MemoryExec { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("MemoryExec") - .field("partitions", &"[...]") - .field("schema", &self.schema) - .field("projection", &self.projection) - .field("sort_information", &self.sort_information) - .finish() + self.inner.fmt_as(DisplayFormatType::Default, f) } } +#[allow(unused, deprecated)] impl DisplayAs for MemoryExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - let partition_sizes: Vec<_> = - self.partitions.iter().map(|b| b.len()).collect(); - - let output_ordering = self - .sort_information - .first() - .map(|output_ordering| { - format!(", output_ordering={}", output_ordering) - }) - .unwrap_or_default(); - - let constraints = self.cache.equivalence_properties().constraints(); - let constraints = if constraints.is_empty() { - String::new() - } else { - format!(", {}", constraints) - }; - - if self.show_sizes { - write!( - f, - "MemoryExec: partitions={}, partition_sizes={partition_sizes:?}{output_ordering}{constraints}", - partition_sizes.len(), - ) - } else { - write!( - f, - "MemoryExec: partitions={}{output_ordering}{constraints}", - partition_sizes.len(), - ) - } - } - } + self.inner.fmt_as(t, f) } } +#[allow(unused, deprecated)] impl ExecutionPlan for MemoryExec { fn name(&self) -> &'static str { "MemoryExec" @@ -129,7 +94,7 @@ impl ExecutionPlan for MemoryExec { } fn properties(&self) -> &PlanProperties { - &self.cache + self.inner.properties() } fn children(&self) -> Vec<&Arc> { @@ -150,18 +115,336 @@ impl ExecutionPlan for MemoryExec { } fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + self.inner.execute(partition, context) + } + + /// We recompute the statistics dynamically from the arrow metadata as it is pretty cheap to do so + fn statistics(&self) -> Result { + self.inner.statistics() + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + self.inner.try_swapping_with_projection(projection) + } +} + +#[allow(unused, deprecated)] +impl MemoryExec { + /// Create a new execution plan for reading in-memory record batches + /// The provided `schema` should not have the projection applied. + pub fn try_new( + partitions: &[Vec], + schema: SchemaRef, + projection: Option>, + ) -> Result { + let source = MemorySourceConfig::try_new(partitions, schema, projection.clone())?; + let data_source = DataSourceExec::new(Arc::new(source)); + Ok(Self { + inner: data_source, + partitions: partitions.to_vec(), + projection, + sort_information: vec![], + show_sizes: true, + }) + } + + /// Create a new execution plan from a list of constant values (`ValuesExec`) + pub fn try_new_as_values( + schema: SchemaRef, + data: Vec>>, + ) -> Result { + if data.is_empty() { + return plan_err!("Values list cannot be empty"); + } + + let n_row = data.len(); + let n_col = schema.fields().len(); + + // We have this single row batch as a placeholder to satisfy evaluation argument + // and generate a single output row + let placeholder_schema = Arc::new(Schema::empty()); + let placeholder_batch = RecordBatch::try_new_with_options( + Arc::clone(&placeholder_schema), + vec![], + &RecordBatchOptions::new().with_row_count(Some(1)), + )?; + + // Evaluate each column + let arrays = (0..n_col) + .map(|j| { + (0..n_row) + .map(|i| { + let expr = &data[i][j]; + let result = expr.evaluate(&placeholder_batch)?; + + match result { + ColumnarValue::Scalar(scalar) => Ok(scalar), + ColumnarValue::Array(array) if array.len() == 1 => { + ScalarValue::try_from_array(&array, 0) + } + ColumnarValue::Array(_) => { + plan_err!("Cannot have array values in a values list") + } + } + }) + .collect::>>() + .and_then(ScalarValue::iter_to_array) + }) + .collect::>>()?; + + let batch = RecordBatch::try_new_with_options( + Arc::clone(&schema), + arrays, + &RecordBatchOptions::new().with_row_count(Some(n_row)), + )?; + + let partitions = vec![batch]; + Self::try_new_from_batches(Arc::clone(&schema), partitions) + } + + /// Create a new plan using the provided schema and batches. + /// + /// Errors if any of the batches don't match the provided schema, or if no + /// batches are provided. + pub fn try_new_from_batches( + schema: SchemaRef, + batches: Vec, + ) -> Result { + if batches.is_empty() { + return plan_err!("Values list cannot be empty"); + } + + for batch in &batches { + let batch_schema = batch.schema(); + if batch_schema != schema { + return plan_err!( + "Batch has invalid schema. Expected: {}, got: {}", + schema, + batch_schema + ); + } + } + + let partitions = vec![batches]; + let source = MemorySourceConfig { + partitions: partitions.clone(), + schema: Arc::clone(&schema), + projected_schema: Arc::clone(&schema), + projection: None, + sort_information: vec![], + show_sizes: true, + fetch: None, + }; + let data_source = DataSourceExec::new(Arc::new(source)); + Ok(Self { + inner: data_source, + partitions, + projection: None, + sort_information: vec![], + show_sizes: true, + }) + } + + fn memory_source_config(&self) -> MemorySourceConfig { + self.inner + .source() + .as_any() + .downcast_ref::() + .unwrap() + .clone() + } + + pub fn with_constraints(mut self, constraints: Constraints) -> Self { + self.inner = self.inner.with_constraints(constraints); + self + } + + /// Set `show_sizes` to determine whether to display partition sizes + pub fn with_show_sizes(mut self, show_sizes: bool) -> Self { + let mut memory_source = self.memory_source_config(); + memory_source.show_sizes = show_sizes; + self.show_sizes = show_sizes; + self.inner = DataSourceExec::new(Arc::new(memory_source)); + self + } + + /// Ref to constraints + pub fn constraints(&self) -> &Constraints { + self.properties().equivalence_properties().constraints() + } + + /// Ref to partitions + pub fn partitions(&self) -> &[Vec] { + &self.partitions + } + + /// Ref to projection + pub fn projection(&self) -> &Option> { + &self.projection + } + + /// Show sizes + pub fn show_sizes(&self) -> bool { + self.show_sizes + } + + /// Ref to sort information + pub fn sort_information(&self) -> &[LexOrdering] { + &self.sort_information + } + + /// A memory table can be ordered by multiple expressions simultaneously. + /// [`EquivalenceProperties`] keeps track of expressions that describe the + /// global ordering of the schema. These columns are not necessarily same; e.g. + /// ```text + /// ┌-------┐ + /// | a | b | + /// |---|---| + /// | 1 | 9 | + /// | 2 | 8 | + /// | 3 | 7 | + /// | 5 | 5 | + /// └---┴---┘ + /// ``` + /// where both `a ASC` and `b DESC` can describe the table ordering. With + /// [`EquivalenceProperties`], we can keep track of these equivalences + /// and treat `a ASC` and `b DESC` as the same ordering requirement. + /// + /// Note that if there is an internal projection, that projection will be + /// also applied to the given `sort_information`. + pub fn try_with_sort_information( + mut self, + sort_information: Vec, + ) -> Result { + self.sort_information = sort_information.clone(); + let mut memory_source = self.memory_source_config(); + memory_source = memory_source.try_with_sort_information(sort_information)?; + self.inner = DataSourceExec::new(Arc::new(memory_source)); + Ok(self) + } + + /// Arc clone of ref to original schema + pub fn original_schema(&self) -> SchemaRef { + Arc::clone(&self.inner.schema()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + constraints: Constraints, + partitions: &[Vec], + ) -> PlanProperties { + PlanProperties::new( + EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints), + Partitioning::UnknownPartitioning(partitions.len()), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +/// Data source configuration for reading in-memory batches of data +#[derive(Clone)] +pub struct MemorySourceConfig { + /// The partitions to query + partitions: Vec>, + /// Schema representing the data before projection + schema: SchemaRef, + /// Schema representing the data after the optional projection is applied + projected_schema: SchemaRef, + /// Optional projection + projection: Option>, + /// Sort information: one or more equivalent orderings + sort_information: Vec, + /// if partition sizes should be displayed + show_sizes: bool, + /// The maximum number of records to read from this plan. If `None`, + /// all records after filtering are returned. + fetch: Option, +} + +impl DataSource for MemorySourceConfig { + fn open( &self, partition: usize, _context: Arc, ) -> Result { - Ok(Box::pin(MemoryStream::try_new( - self.partitions[partition].clone(), + Ok(Box::pin( + MemoryStream::try_new( + self.partitions[partition].clone(), + Arc::clone(&self.projected_schema), + self.projection.clone(), + )? + .with_fetch(self.fetch), + )) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let partition_sizes: Vec<_> = + self.partitions.iter().map(|b| b.len()).collect(); + + let output_ordering = self + .sort_information + .first() + .map(|output_ordering| { + format!(", output_ordering={}", output_ordering) + }) + .unwrap_or_default(); + + let eq_properties = self.eq_properties(); + let constraints = eq_properties.constraints(); + let constraints = if constraints.is_empty() { + String::new() + } else { + format!(", {}", constraints) + }; + + let limit = self + .fetch + .map_or(String::new(), |limit| format!(", fetch={}", limit)); + if self.show_sizes { + write!( + f, + "partitions={}, partition_sizes={partition_sizes:?}{limit}{output_ordering}{constraints}", + partition_sizes.len(), + ) + } else { + write!( + f, + "partitions={}{limit}{output_ordering}{constraints}", + partition_sizes.len(), + ) + } + } + } + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.partitions.len()) + } + + fn eq_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( Arc::clone(&self.projected_schema), - self.projection.clone(), - )?)) + self.sort_information.as_slice(), + ) } - /// We recompute the statistics dynamically from the arrow metadata as it is pretty cheap to do so fn statistics(&self) -> Result { Ok(common::compute_record_batch_statistics( &self.partitions, @@ -170,6 +453,15 @@ impl ExecutionPlan for MemoryExec { )) } + fn with_fetch(&self, limit: Option) -> Option> { + let source = self.clone(); + Some(Arc::new(source.with_limit(limit))) + } + + fn fetch(&self) -> Option { + self.fetch + } + fn try_swapping_with_projection( &self, projection: &ProjectionExec, @@ -178,25 +470,25 @@ impl ExecutionPlan for MemoryExec { // This process can be moved into MemoryExec, but it would be an overlap of their responsibility. all_alias_free_columns(projection.expr()) .then(|| { - let all_projections = (0..self.schema().fields().len()).collect(); + let all_projections = (0..self.schema.fields().len()).collect(); let new_projections = new_projections_for_columns( projection, self.projection().as_ref().unwrap_or(&all_projections), ); - MemoryExec::try_new( + MemorySourceConfig::try_new_exec( self.partitions(), self.original_schema(), Some(new_projections), ) - .map(|e| Arc::new(e) as _) + .map(|e| e as _) }) .transpose() } } -impl MemoryExec { - /// Create a new execution plan for reading in-memory record batches +impl MemorySourceConfig { + /// Create a new `MemorySourceConfig` for reading in-memory record batches /// The provided `schema` should not have the projection applied. pub fn try_new( partitions: &[Vec], @@ -204,29 +496,33 @@ impl MemoryExec { projection: Option>, ) -> Result { let projected_schema = project_schema(&schema, projection.as_ref())?; - let constraints = Constraints::empty(); - let cache = Self::compute_properties( - Arc::clone(&projected_schema), - &[], - constraints, - partitions, - ); Ok(Self { partitions: partitions.to_vec(), schema, projected_schema, projection, sort_information: vec![], - cache, show_sizes: true, + fetch: None, }) } + /// Create a new `DataSourceExec` plan for reading in-memory record batches + /// The provided `schema` should not have the projection applied. + pub fn try_new_exec( + partitions: &[Vec], + schema: SchemaRef, + projection: Option>, + ) -> Result> { + let source = Self::try_new(partitions, schema, projection)?; + Ok(Arc::new(DataSourceExec::new(Arc::new(source)))) + } + /// Create a new execution plan from a list of constant values (`ValuesExec`) pub fn try_new_as_values( schema: SchemaRef, data: Vec>>, - ) -> Result { + ) -> Result> { if data.is_empty() { return plan_err!("Values list cannot be empty"); } @@ -283,7 +579,7 @@ impl MemoryExec { pub fn try_new_from_batches( schema: SchemaRef, batches: Vec, - ) -> Result { + ) -> Result> { if batches.is_empty() { return plan_err!("Values list cannot be empty"); } @@ -300,25 +596,21 @@ impl MemoryExec { } let partitions = vec![batches]; - let cache = Self::compute_properties( - Arc::clone(&schema), - &[], - Constraints::empty(), - &partitions, - ); - Ok(Self { + let source = Self { partitions, schema: Arc::clone(&schema), projected_schema: Arc::clone(&schema), projection: None, sort_information: vec![], - cache, show_sizes: true, - }) + fetch: None, + }; + Ok(Arc::new(DataSourceExec::new(Arc::new(source)))) } - pub fn with_constraints(mut self, constraints: Constraints) -> Self { - self.cache = self.cache.with_constraints(constraints); + /// Set the limit of the files + pub fn with_limit(mut self, limit: Option) -> Self { + self.fetch = limit; self } @@ -328,11 +620,6 @@ impl MemoryExec { self } - /// Ref to constraints - pub fn constraints(&self) -> &Constraints { - self.cache.equivalence_properties().constraints() - } - /// Ref to partitions pub fn partitions(&self) -> &[Vec] { &self.partitions @@ -390,7 +677,7 @@ impl MemoryExec { }); if let Some(col) = ambiguous_column { return internal_err!( - "Column {:?} is not found in the original schema of the MemoryExec", + "Column {:?} is not found in the original schema of the MemorySourceConfig", col ); } @@ -412,19 +699,12 @@ impl MemoryExec { let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &self.original_schema())?; sort_information = base_eqp - .project(&projection_mapping, self.schema()) + .project(&projection_mapping, Arc::clone(&self.projected_schema)) .into_oeq_class() .into_inner(); } self.sort_information = sort_information; - // We need to update equivalence properties when updating sort information. - let eq_properties = EquivalenceProperties::new_with_orderings( - self.schema(), - &self.sort_information, - ); - self.cache = self.cache.with_eq_properties(eq_properties); - Ok(self) } @@ -432,22 +712,6 @@ impl MemoryExec { pub fn original_schema(&self) -> SchemaRef { Arc::clone(&self.schema) } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - orderings: &[LexOrdering], - constraints: Constraints, - partitions: &[Vec], - ) -> PlanProperties { - PlanProperties::new( - EquivalenceProperties::new_with_orderings(schema, orderings) - .with_constraints(constraints), - Partitioning::UnknownPartitioning(partitions.len()), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } } /// Iterator over batches @@ -462,6 +726,8 @@ pub struct MemoryStream { projection: Option>, /// Index into the data index: usize, + /// The remaining number of rows to return + fetch: Option, } impl MemoryStream { @@ -477,6 +743,7 @@ impl MemoryStream { schema, projection, index: 0, + fetch: None, }) } @@ -485,6 +752,12 @@ impl MemoryStream { self.reservation = Some(reservation); self } + + /// Set the number of rows to produce + pub(super) fn with_fetch(mut self, fetch: Option) -> Self { + self.fetch = fetch; + self + } } impl Stream for MemoryStream { @@ -494,20 +767,33 @@ impl Stream for MemoryStream { mut self: std::pin::Pin<&mut Self>, _: &mut Context<'_>, ) -> Poll> { - Poll::Ready(if self.index < self.data.len() { - self.index += 1; - let batch = &self.data[self.index - 1]; + if self.index >= self.data.len() { + return Poll::Ready(None); + } + self.index += 1; + let batch = &self.data[self.index - 1]; + // return just the columns requested + let batch = match self.projection.as_ref() { + Some(columns) => batch.project(columns)?, + None => batch.clone(), + }; + + if self.fetch.is_none() { + return Poll::Ready(Some(Ok(batch))); + } - // return just the columns requested - let batch = match self.projection.as_ref() { - Some(columns) => batch.project(columns)?, - None => batch.clone(), - }; + let fetch = self.fetch.unwrap(); + if fetch == 0 { + return Poll::Ready(None); + } - Some(Ok(batch)) + let batch = if batch.num_rows() > fetch { + batch.slice(0, fetch) } else { - None - }) + batch + }; + self.fetch = Some(fetch - batch.num_rows()); + Poll::Ready(Some(Ok(batch))) } fn size_hint(&self) -> (usize, Option) { @@ -530,7 +816,7 @@ pub trait LazyBatchGenerator: Send + Sync + fmt::Debug + fmt::Display { /// Execution plan for lazy in-memory batches of data /// /// This plan generates output batches lazily, it doesn't have to buffer all batches -/// in memory up front (compared to `MemoryExec`), thus consuming constant memory. +/// in memory up front (compared to `MemorySourceConfig`), thus consuming constant memory. pub struct LazyMemoryExec { /// Schema representing the data schema: SchemaRef, @@ -684,7 +970,8 @@ impl RecordBatchStream for LazyMemoryStream { mod memory_exec_tests { use std::sync::Arc; - use crate::memory::MemoryExec; + use crate::memory::MemorySourceConfig; + use crate::source::DataSourceExec; use crate::ExecutionPlan; use arrow_schema::{DataType, Field, Schema, SortOptions}; @@ -718,8 +1005,10 @@ mod memory_exec_tests { expected_output_order.extend(sort2.clone()); let sort_information = vec![sort1.clone(), sort2.clone()]; - let mem_exec = MemoryExec::try_new(&[vec![]], schema, None)? - .try_with_sort_information(sort_information)?; + let mem_exec = Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&[vec![]], schema, None)? + .try_with_sort_information(sort_information)?, + ))); assert_eq!( mem_exec.properties().output_ordering().unwrap(), @@ -859,12 +1148,40 @@ mod tests { use crate::test::{self, make_partition}; use arrow_schema::{DataType, Field}; + use datafusion_common::assert_batches_eq; use datafusion_common::stats::{ColumnStatistics, Precision}; + use futures::StreamExt; + + #[tokio::test] + async fn exec_with_limit() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let batch = make_partition(7); + let schema = batch.schema(); + let batches = vec![batch.clone(), batch]; + + let exec = MemorySourceConfig::try_new_from_batches(schema, batches).unwrap(); + assert_eq!(exec.fetch(), None); + + let exec = exec.with_fetch(Some(4)).unwrap(); + assert_eq!(exec.fetch(), Some(4)); + + let mut it = exec.execute(0, task_ctx)?; + let mut results = vec![]; + while let Some(batch) = it.next().await { + results.push(batch?); + } + + let expected = [ + "+---+", "| i |", "+---+", "| 0 |", "| 1 |", "| 2 |", "| 3 |", "+---+", + ]; + assert_batches_eq!(expected, &results); + Ok(()) + } #[tokio::test] async fn values_empty_case() -> Result<()> { let schema = test::aggr_test_schema(); - let empty = MemoryExec::try_new_as_values(schema, vec![]); + let empty = MemorySourceConfig::try_new_as_values(schema, vec![]); assert!(empty.is_err()); Ok(()) } @@ -874,14 +1191,14 @@ mod tests { let batch = make_partition(7); let schema = batch.schema(); let batches = vec![batch.clone(), batch]; - let _exec = MemoryExec::try_new_from_batches(schema, batches).unwrap(); + let _exec = MemorySourceConfig::try_new_from_batches(schema, batches).unwrap(); } #[test] fn new_exec_with_batches_empty() { let batch = make_partition(7); let schema = batch.schema(); - let _ = MemoryExec::try_new_from_batches(schema, Vec::new()).unwrap_err(); + let _ = MemorySourceConfig::try_new_from_batches(schema, Vec::new()).unwrap_err(); } #[test] @@ -893,7 +1210,8 @@ mod tests { Field::new("col0", DataType::UInt32, false), Field::new("col1", DataType::Utf8, false), ])); - let _ = MemoryExec::try_new_from_batches(invalid_schema, batches).unwrap_err(); + let _ = MemorySourceConfig::try_new_from_batches(invalid_schema, batches) + .unwrap_err(); } // Test issue: https://github.com/apache/datafusion/issues/8763 @@ -904,10 +1222,13 @@ mod tests { DataType::UInt32, false, )])); - let _ = MemoryExec::try_new_as_values(Arc::clone(&schema), vec![vec![lit(1u32)]]) - .unwrap(); + let _ = MemorySourceConfig::try_new_as_values( + Arc::clone(&schema), + vec![vec![lit(1u32)]], + ) + .unwrap(); // Test that a null value is rejected - let _ = MemoryExec::try_new_as_values( + let _ = MemorySourceConfig::try_new_as_values( schema, vec![vec![lit(ScalarValue::UInt32(None))]], ) @@ -922,7 +1243,7 @@ mod tests { vec![lit(ScalarValue::Null)], ]; let rows = data.len(); - let values = MemoryExec::try_new_as_values( + let values = MemorySourceConfig::try_new_as_values( Arc::new(Schema::new(vec![Field::new("col0", DataType::Null, true)])), data, )?; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index d5429453d00c..63658340f432 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1059,7 +1059,7 @@ mod tests { ErrorExec, MockExec, }, }, - {collect, expressions::col, memory::MemoryExec}, + {collect, expressions::col, memory::MemorySourceConfig}, }; use arrow::array::{ArrayRef, StringArray, UInt32Array}; @@ -1164,8 +1164,12 @@ mod tests { ) -> Result>> { let task_ctx = Arc::new(TaskContext::default()); // create physical plan - let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?; - let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?; + let exec = MemorySourceConfig::try_new_exec( + &input_partitions, + Arc::clone(schema), + None, + )?; + let exec = RepartitionExec::try_new(exec, partitioning)?; // execute and collect results let mut output_partitions = vec![]; @@ -1555,8 +1559,12 @@ mod tests { let task_ctx = Arc::new(task_ctx); // create physical plan - let exec = MemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; - let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?; + let exec = MemorySourceConfig::try_new_exec( + &input_partitions, + Arc::clone(&schema), + None, + )?; + let exec = RepartitionExec::try_new(exec, partitioning)?; // pull partitions for i in 0..exec.partitioning().partition_count() { @@ -1594,13 +1602,14 @@ mod tests { mod test { use arrow_schema::{DataType, Field, Schema, SortOptions}; - use crate::memory::MemoryExec; + use super::*; + use crate::memory::MemorySourceConfig; + use crate::source::DataSourceExec; use crate::union::UnionExec; + use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; - use super::*; - /// Asserts that the plan is as expected /// /// `$EXPECTED_PLAN_LINES`: input plan @@ -1639,8 +1648,8 @@ mod test { let expected_plan = [ "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c0@0 ASC", " UnionExec", - " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", - " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", ]; assert_plan!(expected_plan, exec); Ok(()) @@ -1659,7 +1668,7 @@ mod test { // Repartition should not preserve order let expected_plan = [ "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", ]; assert_plan!(expected_plan, exec); Ok(()) @@ -1681,8 +1690,8 @@ mod test { let expected_plan = [ "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", - " MemoryExec: partitions=1, partition_sizes=[0]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_plan!(expected_plan, exec); Ok(()) @@ -1701,18 +1710,18 @@ mod test { } fn memory_exec(schema: &SchemaRef) -> Arc { - Arc::new(MemoryExec::try_new(&[vec![]], Arc::clone(schema), None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() } fn sorted_memory_exec( schema: &SchemaRef, sort_exprs: LexOrdering, ) -> Arc { - Arc::new( - MemoryExec::try_new(&[vec![]], Arc::clone(schema), None) + Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&[vec![]], Arc::clone(schema), None) .unwrap() .try_with_sort_information(vec![sort_exprs]) .unwrap(), - ) + ))) } } diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index c838376a482e..eeef73c45fc4 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -466,7 +466,7 @@ mod tests { use crate::collect; use crate::expressions::col; use crate::expressions::PhysicalSortExpr; - use crate::memory::MemoryExec; + use crate::memory::MemorySourceConfig; use crate::sorts::sort::SortExec; use crate::test; use crate::test::assert_is_pending; @@ -695,14 +695,13 @@ mod tests { ("c", &(0..100).rev().collect()), ); let schema = batch1.schema(); - Arc::new( - MemoryExec::try_new( - &[vec![batch1, batch2, batch3, batch4]], - Arc::clone(&schema), - None, - ) - .unwrap(), - ) as Arc + + MemorySourceConfig::try_new_exec( + &[vec![batch1, batch2, batch3, batch4]], + Arc::clone(&schema), + None, + ) + .unwrap() as Arc } #[tokio::test] @@ -881,11 +880,8 @@ mod tests { Arc::new(vec![1, 1, 2].into_iter().map(Some).collect::()); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])?; - let input = Arc::new(MemoryExec::try_new( - &[vec![batch]], - Arc::clone(&schema), - None, - )?); + let input = + MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?; let partial_sort_exec = Arc::new(PartialSortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -991,7 +987,7 @@ mod tests { options: option_desc, }, ]), - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None)?), + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None)?, 2, )); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index caa9a754a0d2..2db6e90a1e03 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1069,7 +1069,7 @@ mod tests { use crate::collect; use crate::execution_plan::Boundedness; use crate::expressions::col; - use crate::memory::MemoryExec; + use crate::memory::MemorySourceConfig; use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; @@ -1376,9 +1376,9 @@ mod tests { Arc::new(vec![3, 2, 1].into_iter().map(Some).collect::()); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data]).unwrap(); - let input = Arc::new( - MemoryExec::try_new(&[vec![batch]], Arc::clone(&schema), None).unwrap(), - ); + let input = + MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) + .unwrap(); let sort_exec = Arc::new(SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -1448,11 +1448,7 @@ mod tests { }, }, ]), - Arc::new(MemoryExec::try_new( - &[vec![batch]], - Arc::clone(&schema), - None, - )?), + MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?, )); assert_eq!(DataType::Int32, *sort_exec.schema().field(0).data_type()); @@ -1538,7 +1534,7 @@ mod tests { }, }, ]), - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None)?), + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None)?, )); assert_eq!(DataType::Float32, *sort_exec.schema().field(0).data_type()); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 4f4237204fb1..25572fa60a3d 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -384,7 +384,7 @@ mod tests { use crate::coalesce_partitions::CoalescePartitionsExec; use crate::execution_plan::{Boundedness, EmissionType}; use crate::expressions::col; - use crate::memory::MemoryExec; + use crate::memory::MemorySourceConfig; use crate::metrics::{MetricValue, Timestamp}; use crate::repartition::RepartitionExec; use crate::sorts::sort::SortExec; @@ -450,9 +450,10 @@ mod tests { }, ]); - let exec = MemoryExec::try_new(&[rbs], schema, None).unwrap(); - let repartition_exec = - RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(2))?; + let repartition_exec = RepartitionExec::try_new( + MemorySourceConfig::try_new_exec(&[rbs], schema, None).unwrap(), + Partitioning::RoundRobinBatch(2), + )?; let coalesce_batches_exec = CoalesceBatchesExec::new(Arc::new(repartition_exec), target_batch_size); let spm = SortPreservingMergeExec::new(sort, Arc::new(coalesce_batches_exec)) @@ -542,9 +543,14 @@ mod tests { let schema = batch.schema(); let sort = LexOrdering::default(); // no sort expressions - let exec = MemoryExec::try_new(&[vec![batch.clone()], vec![batch]], schema, None) - .unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec( + &[vec![batch.clone()], vec![batch]], + schema, + None, + ) + .unwrap(); + + let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let res = collect(merge, task_ctx).await.unwrap_err(); assert_contains!( @@ -730,8 +736,8 @@ mod tests { options: Default::default(), }, ]); - let exec = MemoryExec::try_new(partitions, schema, None).unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); + let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, context).await.unwrap(); assert_batches_eq!(exp, collected.as_slice()); @@ -838,9 +844,7 @@ mod tests { let sorted = basic_sort(csv, sort, context).await; let split: Vec<_> = sizes.iter().map(|x| split_batch(&sorted, *x)).collect(); - Ok(Arc::new( - MemoryExec::try_new(&split, sorted.schema(), None).unwrap(), - )) + Ok(MemorySourceConfig::try_new_exec(&split, sorted.schema(), None).unwrap()) } #[tokio::test] @@ -968,8 +972,9 @@ mod tests { }, }, ]); - let exec = MemoryExec::try_new(&[vec![b1], vec![b2]], schema, None).unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) + .unwrap(); + let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); assert_eq!(collected.len(), 1); @@ -1010,10 +1015,10 @@ mod tests { nulls_first: true, }, }]); - let exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap(); - let merge = Arc::new( - SortPreservingMergeExec::new(sort, Arc::new(exec)).with_fetch(Some(2)), - ); + let exec = + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); + let merge = + Arc::new(SortPreservingMergeExec::new(sort, exec).with_fetch(Some(2))); let collected = collect(merge, task_ctx).await.unwrap(); assert_eq!(collected.len(), 1); @@ -1046,8 +1051,9 @@ mod tests { nulls_first: true, }, }]); - let exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); + let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); assert_eq!(collected.len(), 1); @@ -1155,8 +1161,9 @@ mod tests { expr: col("b", &schema).unwrap(), options: Default::default(), }]); - let exec = MemoryExec::try_new(&[vec![b1], vec![b2]], schema, None).unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) + .unwrap(); + let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(Arc::clone(&merge) as Arc, task_ctx) .await @@ -1266,8 +1273,8 @@ mod tests { }, }]); - let exec = MemoryExec::try_new(&partitions, schema, None).unwrap(); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec(&partitions, schema, None).unwrap(); + let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); assert_eq!(collected.len(), 1); diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs new file mode 100644 index 000000000000..0c1dfddd2678 --- /dev/null +++ b/datafusion/physical-plan/src/source.rs @@ -0,0 +1,208 @@ +// 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. + +use std::any::Any; +use std::fmt; +use std::fmt::{Debug, Formatter}; +use std::sync::Arc; + +use crate::execution_plan::{Boundedness, EmissionType}; +use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::projection::ProjectionExec; +use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::{Constraints, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; + +/// Common behaviors in Data Sources for both from Files and Memory. +/// See `DataSourceExec` for physical plan implementation +pub trait DataSource: Send + Sync { + fn open( + &self, + partition: usize, + context: Arc, + ) -> datafusion_common::Result; + fn as_any(&self) -> &dyn Any; + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; + fn repartitioned( + &self, + _target_partitions: usize, + _repartition_file_min_size: usize, + _output_ordering: Option, + ) -> datafusion_common::Result>> { + Ok(None) + } + + fn output_partitioning(&self) -> Partitioning; + fn eq_properties(&self) -> EquivalenceProperties; + fn statistics(&self) -> datafusion_common::Result; + fn with_fetch(&self, _limit: Option) -> Option>; + fn fetch(&self) -> Option; + fn metrics(&self) -> ExecutionPlanMetricsSet { + ExecutionPlanMetricsSet::new() + } + fn try_swapping_with_projection( + &self, + _projection: &ProjectionExec, + ) -> datafusion_common::Result>>; +} + +impl Debug for dyn DataSource { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + write!(f, "DataSource: ") + } +} + +/// Unified data source for file formats like JSON, CSV, AVRO, ARROW, PARQUET +#[derive(Clone, Debug)] +pub struct DataSourceExec { + source: Arc, + cache: PlanProperties, +} + +impl DisplayAs for DataSourceExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + write!(f, "DataSourceExec: ")?; + self.source.fmt_as(t, f) + } +} + +impl ExecutionPlan for DataSourceExec { + fn name(&self) -> &'static str { + "DataSourceExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + Vec::new() + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> datafusion_common::Result> { + Ok(self) + } + + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + ) -> datafusion_common::Result>> { + let source = self.source.repartitioned( + target_partitions, + config.optimizer.repartition_file_min_size, + self.properties().eq_properties.output_ordering(), + )?; + + if let Some(source) = source { + let output_partitioning = source.output_partitioning(); + let plan = self + .clone() + .with_source(source) + // Changing source partitioning may invalidate output partitioning. Update it also + .with_partitioning(output_partitioning); + Ok(Some(Arc::new(plan))) + } else { + Ok(Some(Arc::new(self.clone()))) + } + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> datafusion_common::Result { + self.source.open(partition, context) + } + + fn metrics(&self) -> Option { + Some(self.source.metrics().clone_inner()) + } + + fn statistics(&self) -> datafusion_common::Result { + self.source.statistics() + } + + fn with_fetch(&self, limit: Option) -> Option> { + let mut source = Arc::clone(&self.source); + source = source.with_fetch(limit)?; + let cache = self.cache.clone(); + + Some(Arc::new(Self { source, cache })) + } + + fn fetch(&self) -> Option { + self.source.fetch() + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> datafusion_common::Result>> { + self.source.try_swapping_with_projection(projection) + } +} + +impl DataSourceExec { + pub fn new(source: Arc) -> Self { + let cache = Self::compute_properties(Arc::clone(&source)); + Self { source, cache } + } + + /// Return the source object + pub fn source(&self) -> &Arc { + &self.source + } + + pub fn with_source(mut self, source: Arc) -> Self { + self.cache = Self::compute_properties(Arc::clone(&source)); + self.source = source; + self + } + + /// Assign constraints + pub fn with_constraints(mut self, constraints: Constraints) -> Self { + self.cache = self.cache.with_constraints(constraints); + self + } + + /// Assign output partitioning + pub fn with_partitioning(mut self, partitioning: Partitioning) -> Self { + self.cache = self.cache.with_partitioning(partitioning); + self + } + + fn compute_properties(source: Arc) -> PlanProperties { + PlanProperties::new( + source.eq_properties(), + source.output_partitioning(), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index b7bbfd116954..2d493e27c06a 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -26,7 +26,8 @@ use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use futures::{Future, FutureExt}; -use crate::memory::MemoryExec; +use crate::memory::MemorySourceConfig; +use crate::source::DataSourceExec; use crate::stream::RecordBatchStreamAdapter; use crate::streaming::PartitionStream; use crate::ExecutionPlan; @@ -116,7 +117,7 @@ pub fn build_table_scan_i32( ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } /// Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" @@ -131,18 +132,20 @@ pub fn make_partition(sz: i32) -> RecordBatch { RecordBatch::try_new(schema, vec![arr]).unwrap() } -/// Returns a `MemoryExec` that scans `partitions` of 100 batches each +/// Returns a `DataSourceExec` that scans `partitions` of 100 batches each pub fn scan_partitioned(partitions: usize) -> Arc { Arc::new(mem_exec(partitions)) } -/// Returns a `MemoryExec` that scans `partitions` of 100 batches each -pub fn mem_exec(partitions: usize) -> MemoryExec { +/// Returns a `DataSourceExec` that scans `partitions` of 100 batches each +pub fn mem_exec(partitions: usize) -> DataSourceExec { let data: Vec> = (0..partitions).map(|_| vec![make_partition(100)]).collect(); let schema = data[0][0].schema(); let projection = None; - MemoryExec::try_new(&data, schema, projection).unwrap() + DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&data, schema, projection).unwrap(), + )) } // Construct a stream partition for test purposes diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 91d2f2c9e869..7e0f88784644 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -640,9 +640,10 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { mod tests { use super::*; use crate::collect; - use crate::memory::MemoryExec; + use crate::memory::MemorySourceConfig; use crate::test; + use crate::source::DataSourceExec; use arrow_schema::{DataType, SortOptions}; use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::col; @@ -863,14 +864,14 @@ mod tests { .iter() .map(|ordering| convert_to_sort_exprs(ordering)) .collect::>(); - let child1 = Arc::new( - MemoryExec::try_new(&[], Arc::clone(&schema), None)? + let child1 = Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(first_orderings)?, - ); - let child2 = Arc::new( - MemoryExec::try_new(&[], Arc::clone(&schema), None)? + ))); + let child2 = Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(second_orderings)?, - ); + ))); let mut union_expected_eq = EquivalenceProperties::new(Arc::clone(&schema)); union_expected_eq.add_new_orderings(union_expected_orderings); diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 960e3f544ee0..ea1086c0a3d6 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -34,16 +34,10 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; /// Execution plan for values list based relation (produces constant rows) -/// -/// Note this structure is the same as [`MemoryExec`] and is deprecated. -/// Please see the following for alternatives -/// * [`MemoryExec::try_new`] -/// * [`MemoryExec::try_new_from_batches`] -/// -/// [`MemoryExec`]: crate::memory::MemoryExec -/// [`MemoryExec::try_new`]: crate::memory::MemoryExec::try_new -/// [`MemoryExec::try_new_from_batches`]: crate::memory::MemoryExec::try_new_from_batches -#[deprecated(since = "45.0.0", note = "Use `MemoryExec` instead")] +#[deprecated( + since = "45.0.0", + note = "Use `MemorySourceConfig::try_new_as_values` instead" +)] #[derive(Debug, Clone)] pub struct ValuesExec { /// The schema diff --git a/datafusion/physical-plan/src/visitor.rs b/datafusion/physical-plan/src/visitor.rs index ca826c50022d..892e603a016d 100644 --- a/datafusion/physical-plan/src/visitor.rs +++ b/datafusion/physical-plan/src/visitor.rs @@ -47,15 +47,15 @@ pub fn accept( /// ```text /// ProjectionExec: id /// FilterExec: state = CO -/// CsvExec: +/// DataSourceExec: /// ``` /// /// The sequence of visit operations would be: /// ```text /// visitor.pre_visit(ProjectionExec) /// visitor.pre_visit(FilterExec) -/// visitor.pre_visit(CsvExec) -/// visitor.post_visit(CsvExec) +/// visitor.pre_visit(DataSourceExec) +/// visitor.post_visit(DataSourceExec) /// visitor.post_visit(FilterExec) /// visitor.post_visit(ProjectionExec) /// ``` diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index cf19d0daf378..41de16289cf5 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1171,7 +1171,9 @@ mod tests { use std::task::{Context, Poll}; use std::time::Duration; + use crate::common::collect; use crate::expressions::PhysicalSortExpr; + use crate::memory::MemorySourceConfig; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; use crate::windows::{ @@ -1196,11 +1198,9 @@ mod tests { use datafusion_functions_window::nth_value::last_value_udwf; use datafusion_functions_window::nth_value::nth_value_udwf; use datafusion_physical_expr::expressions::{col, Column, Literal}; + use datafusion_physical_expr::window::StandardWindowExpr; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; - use crate::common::collect; - use crate::memory::MemoryExec; - use datafusion_physical_expr::window::StandardWindowExpr; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; use itertools::Itertools; @@ -1531,12 +1531,11 @@ mod tests { vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], )?; - let memory_exec = MemoryExec::try_new( + let memory_exec = MemorySourceConfig::try_new_exec( &[vec![batch.clone(), batch.clone(), batch.clone()]], Arc::clone(&schema), None, - ) - .map(|e| Arc::new(e) as Arc)?; + )?; let col_a = col("a", &schema)?; let nth_value_func1 = create_udwf_window_expr( &nth_value_udwf(), @@ -1618,7 +1617,7 @@ mod tests { let expected = vec![ "BoundedWindowAggExec: wdw=[last: Ok(Field { name: \"last\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-1): Ok(Field { name: \"nth_value(-1)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }, nth_value(-2): Ok(Field { name: \"nth_value(-2)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted]", - " MemoryExec: partitions=1, partition_sizes=[3]", + " DataSourceExec: partitions=1, partition_sizes=[3]", ]; // Get string representation of the plan let actual = get_plan_string(&physical_plan); diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index e2dbb39ca186..34fb5bb6ddc1 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -26,6 +26,7 @@ use object_store::path::Path; use object_store::ObjectMeta; use datafusion::arrow::datatypes::Schema; +use datafusion::datasource::data_source::FileSource; use datafusion::datasource::file_format::csv::CsvSink; use datafusion::datasource::file_format::json::JsonSink; #[cfg(feature = "parquet")] @@ -469,12 +470,19 @@ pub fn parse_protobuf_partitioning( } } +pub fn parse_protobuf_file_scan_schema( + proto: &protobuf::FileScanExecConf, +) -> Result> { + Ok(Arc::new(convert_required!(proto.schema)?)) +} + pub fn parse_protobuf_file_scan_config( proto: &protobuf::FileScanExecConf, registry: &dyn FunctionRegistry, codec: &dyn PhysicalExtensionCodec, + source: Arc, ) -> Result { - let schema: Arc = Arc::new(convert_required!(proto.schema)?); + let schema: Arc = parse_protobuf_file_scan_schema(proto)?; let projection = proto .projection .iter() @@ -530,17 +538,14 @@ pub fn parse_protobuf_file_scan_config( output_ordering.push(sort_expr); } - Ok(FileScanConfig { - object_store_url, - file_schema, - file_groups, - constraints, - statistics, - projection, - limit: proto.limit.as_ref().map(|sl| sl.limit as usize), - table_partition_cols, - output_ordering, - }) + Ok(FileScanConfig::new(object_store_url, file_schema, source) + .with_file_groups(file_groups) + .with_constraints(constraints) + .with_statistics(statistics) + .with_projection(projection) + .with_limit(proto.limit.as_ref().map(|sl| sl.limit as usize)) + .with_table_partition_cols(table_partition_cols) + .with_output_ordering(output_ordering)) } impl TryFrom<&protobuf::PartitionedFile> for PartitionedFile { diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 9ded9122b7da..6fa28e882ed6 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -30,8 +30,8 @@ use datafusion::datasource::file_format::json::JsonSink; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; #[cfg(feature = "parquet")] -use datafusion::datasource::physical_plan::ParquetExec; -use datafusion::datasource::physical_plan::{AvroExec, CsvExec}; +use datafusion::datasource::physical_plan::ParquetSource; +use datafusion::datasource::physical_plan::{AvroSource, CsvSource, FileScanConfig}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::aggregate::AggregateFunctionExpr; @@ -57,12 +57,14 @@ use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::unnest::{ListUnnest, UnnestExec}; use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion::physical_plan::{ ExecutionPlan, InputOrderMode, PhysicalExpr, WindowExpr, }; +use datafusion_common::config::TableParquetOptions; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; @@ -70,6 +72,7 @@ use crate::common::{byte_to_string, str_to_byte}; use crate::physical_plan::from_proto::{ parse_physical_expr, parse_physical_sort_expr, parse_physical_sort_exprs, parse_physical_window_expr, parse_protobuf_file_scan_config, + parse_protobuf_file_scan_schema, }; use crate::physical_plan::to_proto::{ serialize_file_scan_config, serialize_maybe_filter, serialize_physical_aggr_expr, @@ -203,47 +206,51 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { )), } } - PhysicalPlanType::CsvScan(scan) => Ok(Arc::new( - CsvExec::builder(parse_protobuf_file_scan_config( + PhysicalPlanType::CsvScan(scan) => { + let escape = if let Some( + protobuf::csv_scan_exec_node::OptionalEscape::Escape(escape), + ) = &scan.optional_escape + { + Some(str_to_byte(escape, "escape")?) + } else { + None + }; + + let comment = if let Some( + protobuf::csv_scan_exec_node::OptionalComment::Comment(comment), + ) = &scan.optional_comment + { + Some(str_to_byte(comment, "comment")?) + } else { + None + }; + + let source = Arc::new( + CsvSource::new( + scan.has_header, + str_to_byte(&scan.delimiter, "delimiter")?, + 0, + ) + .with_escape(escape) + .with_comment(comment), + ); + + let conf = parse_protobuf_file_scan_config( scan.base_conf.as_ref().unwrap(), registry, extension_codec, - )?) - .with_has_header(scan.has_header) - .with_delimeter(str_to_byte(&scan.delimiter, "delimiter")?) - .with_quote(str_to_byte(&scan.quote, "quote")?) - .with_escape( - if let Some(protobuf::csv_scan_exec_node::OptionalEscape::Escape( - escape, - )) = &scan.optional_escape - { - Some(str_to_byte(escape, "escape")?) - } else { - None - }, - ) - .with_comment( - if let Some(protobuf::csv_scan_exec_node::OptionalComment::Comment( - comment, - )) = &scan.optional_comment - { - Some(str_to_byte(comment, "comment")?) - } else { - None - }, - ) + source, + )? .with_newlines_in_values(scan.newlines_in_values) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - )), + .with_file_compression_type(FileCompressionType::UNCOMPRESSED); + Ok(conf.new_exec()) + } #[cfg_attr(not(feature = "parquet"), allow(unused_variables))] PhysicalPlanType::ParquetScan(scan) => { #[cfg(feature = "parquet")] { - let base_config = parse_protobuf_file_scan_config( + let schema = parse_protobuf_file_scan_schema( scan.base_conf.as_ref().unwrap(), - registry, - extension_codec, )?; let predicate = scan .predicate @@ -252,31 +259,40 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { parse_physical_expr( expr, registry, - base_config.file_schema.as_ref(), + schema.as_ref(), extension_codec, ) }) .transpose()?; - let mut builder = ParquetExec::builder(base_config); + let mut options = TableParquetOptions::default(); - if let Some(options) = scan.parquet_options.as_ref() { - builder = builder.with_table_parquet_options(options.try_into()?) + if let Some(table_options) = scan.parquet_options.as_ref() { + options = table_options.try_into()?; } + let mut source = ParquetSource::new(options); if let Some(predicate) = predicate { - builder = builder.with_predicate(predicate) + source = source.with_predicate(Arc::clone(&schema), predicate); } - Ok(builder.build_arc()) + let base_config = parse_protobuf_file_scan_config( + scan.base_conf.as_ref().unwrap(), + registry, + extension_codec, + Arc::new(source), + )?; + Ok(base_config.new_exec()) } #[cfg(not(feature = "parquet"))] panic!("Unable to process a Parquet PhysicalPlan when `parquet` feature is not enabled") } PhysicalPlanType::AvroScan(scan) => { - Ok(Arc::new(AvroExec::new(parse_protobuf_file_scan_config( + let conf = parse_protobuf_file_scan_config( scan.base_conf.as_ref().unwrap(), registry, extension_codec, - )?))) + Arc::new(AvroSource::new()), + )?; + Ok(conf.new_exec()) } PhysicalPlanType::CoalesceBatches(coalesce_batches) => { let input: Arc = into_physical_plan( @@ -1614,68 +1630,97 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { }); } - if let Some(exec) = plan.downcast_ref::() { - return Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::CsvScan( - protobuf::CsvScanExecNode { - base_conf: Some(serialize_file_scan_config( - exec.base_config(), - extension_codec, - )?), - has_header: exec.has_header(), - delimiter: byte_to_string(exec.delimiter(), "delimiter")?, - quote: byte_to_string(exec.quote(), "quote")?, - optional_escape: if let Some(escape) = exec.escape() { - Some(protobuf::csv_scan_exec_node::OptionalEscape::Escape( - byte_to_string(escape, "escape")?, - )) - } else { - None - }, - optional_comment: if let Some(comment) = exec.comment() { - Some(protobuf::csv_scan_exec_node::OptionalComment::Comment( - byte_to_string(comment, "comment")?, - )) - } else { - None - }, - newlines_in_values: exec.newlines_in_values(), - }, - )), - }); + if let Some(exec) = plan.downcast_ref::() { + let source = exec.source(); + if let Some(maybe_csv) = source.as_any().downcast_ref::() { + let source = maybe_csv.file_source(); + if let Some(csv_config) = source.as_any().downcast_ref::() { + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::CsvScan( + protobuf::CsvScanExecNode { + base_conf: Some(serialize_file_scan_config( + maybe_csv, + extension_codec, + )?), + has_header: csv_config.has_header(), + delimiter: byte_to_string( + csv_config.delimiter(), + "delimiter", + )?, + quote: byte_to_string(csv_config.quote(), "quote")?, + optional_escape: if let Some(escape) = csv_config.escape() + { + Some( + protobuf::csv_scan_exec_node::OptionalEscape::Escape( + byte_to_string(escape, "escape")?, + ), + ) + } else { + None + }, + optional_comment: if let Some(comment) = + csv_config.comment() + { + Some(protobuf::csv_scan_exec_node::OptionalComment::Comment( + byte_to_string(comment, "comment")?, + )) + } else { + None + }, + newlines_in_values: maybe_csv.newlines_in_values(), + }, + )), + }); + } + } } #[cfg(feature = "parquet")] - if let Some(exec) = plan.downcast_ref::() { - let predicate = exec - .predicate() - .map(|pred| serialize_physical_expr(pred, extension_codec)) - .transpose()?; - return Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::ParquetScan( - protobuf::ParquetScanExecNode { - base_conf: Some(serialize_file_scan_config( - exec.base_config(), - extension_codec, - )?), - predicate, - parquet_options: Some(exec.table_parquet_options().try_into()?), - }, - )), - }); + if let Some(exec) = plan.downcast_ref::() { + let source = exec.source(); + if let Some(maybe_parquet) = source.as_any().downcast_ref::() + { + let source = maybe_parquet.file_source(); + if let Some(conf) = source.as_any().downcast_ref::() { + let predicate = conf + .predicate() + .map(|pred| serialize_physical_expr(pred, extension_codec)) + .transpose()?; + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::ParquetScan( + protobuf::ParquetScanExecNode { + base_conf: Some(serialize_file_scan_config( + maybe_parquet, + extension_codec, + )?), + predicate, + parquet_options: Some( + conf.table_parquet_options().try_into()?, + ), + }, + )), + }); + } + } } - if let Some(exec) = plan.downcast_ref::() { - return Ok(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::AvroScan( - protobuf::AvroScanExecNode { - base_conf: Some(serialize_file_scan_config( - exec.base_config(), - extension_codec, - )?), - }, - )), - }); + if let Some(exec) = plan.downcast_ref::() { + let source = exec.source(); + if let Some(maybe_avro) = source.as_any().downcast_ref::() { + let source = maybe_avro.file_source(); + if source.as_any().downcast_ref::().is_some() { + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::AvroScan( + protobuf::AvroScanExecNode { + base_conf: Some(serialize_file_scan_config( + maybe_avro, + extension_codec, + )?), + }, + )), + }); + } + } } if let Some(exec) = plan.downcast_ref::() { diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 50c08024464f..fdd529cfd1b9 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -41,13 +41,14 @@ use datafusion::arrow::compute::kernels::sort::SortOptions; use datafusion::arrow::datatypes::{DataType, Field, IntervalUnit, Schema}; use datafusion::datasource::empty::EmptyTable; use datafusion::datasource::file_format::csv::CsvSink; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::file_format::json::JsonSink; use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::datasource::listing::{ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, - FileSinkConfig, ParquetExec, + FileSinkConfig, ParquetSource, }; use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::sum::sum_udaf; @@ -700,13 +701,25 @@ fn roundtrip_coalesce_with_fetch() -> Result<()> { #[test] fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("col", DataType::Utf8, false)])); + + let predicate = Arc::new(BinaryExpr::new( + Arc::new(Column::new("col", 1)), + Operator::Eq, + lit("1"), + )); + + let mut options = TableParquetOptions::new(); + options.global.pushdown_filters = true; + + let source = Arc::new( + ParquetSource::new(options).with_predicate(Arc::clone(&file_schema), predicate), + ); + let scan_config = FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), - file_schema: Arc::new(Schema::new(vec![Field::new( - "col", - DataType::Utf8, - false, - )])), + file_schema, file_groups: vec![vec![PartitionedFile::new( "/path/to/file.parquet".to_string(), 1024, @@ -723,23 +736,12 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { limit: None, table_partition_cols: vec![], output_ordering: vec![], + file_compression_type: FileCompressionType::UNCOMPRESSED, + new_lines_in_values: false, + source, }; - let predicate = Arc::new(BinaryExpr::new( - Arc::new(Column::new("col", 1)), - Operator::Eq, - lit("1"), - )); - - let mut options = TableParquetOptions::new(); - options.global.pushdown_filters = true; - - roundtrip_test( - ParquetExec::builder(scan_config) - .with_predicate(predicate) - .with_table_parquet_options(options) - .build_arc(), - ) + roundtrip_test(scan_config.new_exec()) } #[tokio::test] @@ -750,6 +752,7 @@ async fn roundtrip_parquet_exec_with_table_partition_cols() -> Result<()> { vec![wrap_partition_value_in_dict(ScalarValue::Int64(Some(0)))]; let schema = Arc::new(Schema::new(vec![Field::new("col", DataType::Utf8, false)])); + let source = Arc::new(ParquetSource::default()); let scan_config = FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![file_group]], @@ -764,20 +767,31 @@ async fn roundtrip_parquet_exec_with_table_partition_cols() -> Result<()> { false, )], output_ordering: vec![], + file_compression_type: FileCompressionType::UNCOMPRESSED, + new_lines_in_values: false, + source, }; - roundtrip_test(ParquetExec::builder(scan_config).build_arc()) + roundtrip_test(scan_config.new_exec()) } #[test] fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { + let file_schema = + Arc::new(Schema::new(vec![Field::new("col", DataType::Utf8, false)])); + + let custom_predicate_expr = Arc::new(CustomPredicateExpr { + inner: Arc::new(Column::new("col", 1)), + }); + + let source = Arc::new( + ParquetSource::default() + .with_predicate(Arc::clone(&file_schema), custom_predicate_expr), + ); + let scan_config = FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), - file_schema: Arc::new(Schema::new(vec![Field::new( - "col", - DataType::Utf8, - false, - )])), + file_schema, file_groups: vec![vec![PartitionedFile::new( "/path/to/file.parquet".to_string(), 1024, @@ -794,6 +808,9 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { limit: None, table_partition_cols: vec![], output_ordering: vec![], + file_compression_type: FileCompressionType::UNCOMPRESSED, + new_lines_in_values: false, + source, }; #[derive(Debug, Clone, Eq)] @@ -901,12 +918,7 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { } } - let custom_predicate_expr = Arc::new(CustomPredicateExpr { - inner: Arc::new(Column::new("col", 1)), - }); - let exec_plan = ParquetExec::builder(scan_config) - .with_predicate(custom_predicate_expr) - .build_arc(); + let exec_plan = scan_config.new_exec(); let ctx = SessionContext::new(); roundtrip_test_and_return(exec_plan, &ctx, &CustomPhysicalExtensionCodec {})?; diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index 09cf70280e7c..a9325e452ae8 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -120,13 +120,13 @@ fn expand_row(mut row: Vec) -> impl Iterator> { /// normalize path references /// /// ```text -/// CsvExec: files={1 group: [[path/to/datafusion/testing/data/csv/aggregate_test_100.csv]]}, ... +/// DataSourceExec: files={1 group: [[path/to/datafusion/testing/data/csv/aggregate_test_100.csv]]}, ... /// ``` /// /// into: /// /// ```text -/// CsvExec: files={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, ... +/// DataSourceExec: files={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, ... /// ``` fn normalize_paths(mut row: Vec) -> Vec { row.iter_mut().for_each(|s| { diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index 9a0a1d587433..9aeaaacb1071 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -50,7 +50,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], file_type=csv, has_header=true query TT @@ -70,7 +70,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], file_type=csv, has_header=true query TT EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1 + 100] as result @@ -89,7 +89,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], file_type=csv, has_header=true query II SELECT a, ARRAY_AGG(c ORDER BY c)[1] as result diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 9d7ece1edd6e..d6a8c428ac46 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -204,7 +204,7 @@ physical_plan 03)----AggregateExec: mode=Partial, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] 04)------SortExec: expr=[c2@1 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[true] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true # test array_agg_order with list data type statement ok @@ -1058,7 +1058,7 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; @@ -5042,7 +5042,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true # @@ -5067,7 +5067,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 group by c3 limit 5; @@ -5091,7 +5091,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 group by c2, c3 limit 5 offset 4; @@ -5126,7 +5126,7 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query I SELECT DISTINCT c3 FROM aggregate_test_100 WHERE c3 between 10 and 20 group by c2, c3 limit 4; @@ -5152,7 +5152,7 @@ physical_plan 04)------CoalescePartitionsExec 05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[max(aggregate_test_100.c1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true # TODO(msirek): Extend checking in LimitedDistinctAggregation equal groupings to ignore the order of columns # in the group-by column lists, so the limit could be pushed to the lowest AggregateExec in this case @@ -5176,7 +5176,7 @@ physical_plan 08)--------------CoalescePartitionsExec 09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c2, c3 limit 3 offset 10; @@ -5200,7 +5200,7 @@ physical_plan 04)------CoalescePartitionsExec 05)--------AggregateExec: mode=Partial, gby=[(NULL as c2, NULL as c3), (c2@0 as c2, NULL as c3), (c2@0 as c2, c3@1 as c3)], aggr=[] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 group by rollup(c2, c3) limit 3; @@ -5227,7 +5227,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], file_type=csv, has_header=true statement ok set datafusion.optimizer.enable_distinct_aggregation_soft_limit = true; @@ -6012,7 +6012,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 ASC NULLS LAST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], file_type=csv, has_header=true # test last to first query TT @@ -6026,7 +6026,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 DESC NULLS FIRST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], file_type=csv, has_header=true # test building plan with aggreagte sum @@ -6099,7 +6099,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query error @@ -6274,7 +6274,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(aggregate_test_100.c5)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true ####### # Group median test diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index a67fec695f6c..5fa0845cd2d5 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -51,7 +51,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TI select * from (select trace_id, MAX(timestamp) max_ts from traces t group by trace_id) where trace_id != 'b' order by max_ts desc limit 3; @@ -115,7 +115,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) desc limit 4; @@ -132,7 +132,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) asc limit 4; @@ -149,7 +149,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by trace_id asc limit 4; @@ -166,7 +166,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TI select trace_id, max(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; diff --git a/datafusion/sqllogictest/test_files/arrow_files.slt b/datafusion/sqllogictest/test_files/arrow_files.slt index e73acc384cb3..30f322cf98fc 100644 --- a/datafusion/sqllogictest/test_files/arrow_files.slt +++ b/datafusion/sqllogictest/test_files/arrow_files.slt @@ -32,7 +32,7 @@ query TT EXPLAIN SELECT * FROM arrow_simple ---- logical_plan TableScan: arrow_simple projection=[f0, f1, f2] -physical_plan ArrowExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[f0, f1, f2] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[f0, f1, f2], file_type=arrow # correct content query ITB @@ -117,7 +117,7 @@ query TT EXPLAIN SELECT f0 FROM arrow_partitioned WHERE part = 456 ---- logical_plan TableScan: arrow_partitioned projection=[f0], full_filters=[arrow_partitioned.part = Int32(456)] -physical_plan ArrowExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_arrow/part=456/data.arrow]]}, projection=[f0] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_arrow/part=456/data.arrow]]}, projection=[f0], file_type=arrow # Errors in partition filters should be reported diff --git a/datafusion/sqllogictest/test_files/avro.slt b/datafusion/sqllogictest/test_files/avro.slt index d6323b88f159..80bf0bc2dd5a 100644 --- a/datafusion/sqllogictest/test_files/avro.slt +++ b/datafusion/sqllogictest/test_files/avro.slt @@ -250,4 +250,4 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]} +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/alltypes_plain.avro]]}, file_type=avro diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index caa708483a11..cd0a38a5e007 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -172,7 +172,7 @@ logical_plan 02)--TableScan: source_table projection=[col1, col2] physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Error case query error DataFusion error: Invalid or Unsupported Configuration: Format not explicitly set and unable to get file extension! Use STORED AS to define file format. @@ -186,7 +186,7 @@ logical_plan 02)--TableScan: source_table projection=[col1, col2] physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Copy to directory as partitioned files with keep_partition_by_columns enabled query I diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 3625da68b39e..d660257b609d 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -50,7 +50,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 0; @@ -69,7 +69,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; @@ -87,7 +87,7 @@ physical_plan 01)ProjectionExec: expr=[a@0 as a, count() PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as count_a] 02)--WindowAggExec: wdw=[count() PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count() PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT a, COUNT() OVER (PARTITION BY a) AS count_a FROM t1 ORDER BY a; diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index 8328f9bbd295..bb66aef2514c 100644 --- a/datafusion/sqllogictest/test_files/create_external_table.slt +++ b/datafusion/sqllogictest/test_files/create_external_table.slt @@ -253,7 +253,7 @@ EXPLAIN SELECT id FROM t ORDER BY id ASC; logical_plan 01)Sort: t.id ASC NULLS LAST 02)--TableScan: t projection=[id] -physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet ## Test a DESC order and verify that output_ordering is ASC from the previous OBRDER BY query TT @@ -264,7 +264,7 @@ logical_plan 02)--TableScan: t projection=[id] physical_plan 01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet statement ok DROP TABLE t; @@ -278,7 +278,7 @@ query TT EXPLAIN SELECT id FROM t; ---- logical_plan TableScan: t projection=[id] -physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC], file_type=parquet statement ok DROP TABLE t; diff --git a/datafusion/sqllogictest/test_files/cse.slt b/datafusion/sqllogictest/test_files/cse.slt index c95e9a1309f8..1af4f14c937e 100644 --- a/datafusion/sqllogictest/test_files/cse.slt +++ b/datafusion/sqllogictest/test_files/cse.slt @@ -32,7 +32,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as c1, __common_expr_1@0 as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1] -03)----MemoryExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Common volatile expression query TT @@ -46,7 +46,7 @@ logical_plan 02)--TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[a@0 + random() as c1, a@0 + random() as c2] -02)--MemoryExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] # Volatile expression with non-volatile common child query TT @@ -62,7 +62,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 + random() as c1, __common_expr_1@0 + random() as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1] -03)----MemoryExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Volatile expression with non-volatile common children query TT @@ -78,7 +78,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 + random() + __common_expr_2@1 as c1, __common_expr_1@0 + random() + __common_expr_2@1 as c2] 02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1, a@0 + 2 as __common_expr_2] -03)----MemoryExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Common short-circuit expression query TT @@ -100,7 +100,7 @@ physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as c1, __common_expr_1@0 as c2, __common_expr_2@1 as c3, __common_expr_2@1 as c4, __common_expr_3@2 as c5, __common_expr_3@2 as c6] 02)--ProjectionExec: expr=[__common_expr_4@0 AND b@1 = 0 as __common_expr_1, __common_expr_4@0 OR b@1 = 0 as __common_expr_2, CASE WHEN __common_expr_4@0 THEN 0 ELSE 1 END as __common_expr_3] 03)----ProjectionExec: expr=[a@0 = 0 as __common_expr_4, b@1 as b] -04)------MemoryExec: partitions=1, partition_sizes=[0] +04)------DataSourceExec: partitions=1, partition_sizes=[0] # Common children of short-circuit expression query TT @@ -128,7 +128,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND b@4 = 0 as c1, __common_expr_1@0 AND b@4 = 1 as c2, b@4 = 2 AND a@3 = 1 as c3, b@4 = 3 AND a@3 = 1 as c4, __common_expr_2@1 OR b@4 = 4 as c5, __common_expr_2@1 OR b@4 = 5 as c6, b@4 = 6 OR a@3 = 3 as c7, b@4 = 7 OR a@3 = 3 as c8, CASE WHEN __common_expr_3@2 THEN 0 ELSE 1 END as c9, CASE WHEN __common_expr_3@2 THEN 0 ELSE 2 END as c10, CASE WHEN b@4 = 8 THEN a@3 + 1 ELSE 0 END as c11, CASE WHEN b@4 = 9 THEN a@3 + 1 ELSE 0 END as c12, CASE WHEN b@4 = 10 THEN 0 ELSE a@3 + 2 END as c13, CASE WHEN b@4 = 11 THEN 0 ELSE a@3 + 2 END as c14] 02)--ProjectionExec: expr=[a@0 = 0 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 = 4 as __common_expr_3, a@0 as a, b@1 as b] -03)----MemoryExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Common children of volatile, short-circuit expression query TT @@ -156,7 +156,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND b@4 = random() as c1, __common_expr_1@0 AND b@4 = 1 + random() as c2, b@4 = 2 + random() AND a@3 = 1 as c3, b@4 = 3 + random() AND a@3 = 1 as c4, __common_expr_2@1 OR b@4 = 4 + random() as c5, __common_expr_2@1 OR b@4 = 5 + random() as c6, b@4 = 6 + random() OR a@3 = 3 as c7, b@4 = 7 + random() OR a@3 = 3 as c8, CASE WHEN __common_expr_3@2 THEN random() ELSE 1 END as c9, CASE WHEN __common_expr_3@2 THEN random() ELSE 2 END as c10, CASE WHEN b@4 = 8 + random() THEN a@3 + 1 ELSE 0 END as c11, CASE WHEN b@4 = 9 + random() THEN a@3 + 1 ELSE 0 END as c12, CASE WHEN b@4 = 10 + random() THEN 0 ELSE a@3 + 2 END as c13, CASE WHEN b@4 = 11 + random() THEN 0 ELSE a@3 + 2 END as c14] 02)--ProjectionExec: expr=[a@0 = 0 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 = 4 as __common_expr_3, a@0 as a, b@1 as b] -03)----MemoryExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Common volatile children of short-circuit expression query TT @@ -174,7 +174,7 @@ logical_plan 02)--TableScan: t1 projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 = random() AND b@1 = 0 as c1, a@0 = random() AND b@1 = 1 as c2, a@0 = 2 + random() OR b@1 = 4 as c3, a@0 = 2 + random() OR b@1 = 5 as c4, CASE WHEN a@0 = 4 + random() THEN 0 ELSE 1 END as c5, CASE WHEN a@0 = 4 + random() THEN 0 ELSE 2 END as c6] -02)--MemoryExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] # Surely only once but also conditionally evaluated expressions query TT @@ -194,7 +194,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[(__common_expr_1@0 OR random() = 0) AND __common_expr_2@1 as c1, __common_expr_2@1 AND random() = 0 OR __common_expr_1@0 as c2, CASE WHEN __common_expr_3@2 = 0 THEN __common_expr_3@2 ELSE 0 END as c3, CASE WHEN __common_expr_4@3 = 0 THEN 0 WHEN CAST(__common_expr_4@3 AS Boolean) THEN 0 ELSE 0 END as c4, CASE WHEN __common_expr_5@4 = 0 THEN 0 WHEN random() = 0 THEN __common_expr_5@4 ELSE 0 END as c5, CASE WHEN __common_expr_6@5 = 0 THEN 0 ELSE __common_expr_6@5 END as c6] 02)--ProjectionExec: expr=[a@0 = 1 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 + 3 as __common_expr_3, a@0 + 4 as __common_expr_4, a@0 + 5 as __common_expr_5, a@0 + 6 as __common_expr_6] -03)----MemoryExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Surely only once but also conditionally evaluated subexpressions query TT @@ -212,7 +212,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[(__common_expr_1@0 OR random() = 0) AND (__common_expr_2@1 OR random() = 1) as c1, __common_expr_2@1 AND random() = 0 OR __common_expr_1@0 AND random() = 1 as c2, CASE WHEN __common_expr_3@2 = 0 THEN __common_expr_3@2 + random() ELSE 0 END as c3, CASE WHEN __common_expr_4@3 = 0 THEN 0 ELSE __common_expr_4@3 + random() END as c4] 02)--ProjectionExec: expr=[a@0 = 1 as __common_expr_1, a@0 = 2 as __common_expr_2, a@0 + 3 as __common_expr_3, a@0 + 4 as __common_expr_4] -03)----MemoryExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] # Only conditionally evaluated expressions query TT @@ -230,4 +230,4 @@ logical_plan 02)--TableScan: t1 projection=[a] physical_plan 01)ProjectionExec: expr=[(random() = 0 OR a@0 = 1) AND a@0 = 2 as c1, random() = 0 AND a@0 = 2 OR a@0 = 1 as c2, CASE WHEN random() = 0 THEN a@0 + 3 ELSE a@0 + 3 END as c3, CASE WHEN random() = 0 THEN 0 WHEN a@0 + 4 = 0 THEN a@0 + 4 ELSE 0 END as c4, CASE WHEN random() = 0 THEN 0 WHEN a@0 + 5 = 0 THEN 0 ELSE a@0 + 5 END as c5, CASE WHEN random() = 0 THEN 0 WHEN random() = 0 THEN a@0 + 6 ELSE a@0 + 6 END as c6] -02)--MemoryExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] diff --git a/datafusion/sqllogictest/test_files/csv_files.slt b/datafusion/sqllogictest/test_files/csv_files.slt index 5906c6a19bb8..5a7fa309dbfa 100644 --- a/datafusion/sqllogictest/test_files/csv_files.slt +++ b/datafusion/sqllogictest/test_files/csv_files.slt @@ -165,7 +165,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [int_col@0 ASC NULLS LAST] 02)--SortExec: expr=[int_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CsvExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/csv_files/csv_partitions/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/csv_files/csv_partitions/2.csv]]}, projection=[int_col, string_col, bigint_col, partition_col], has_header=false +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/csv_files/csv_partitions/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/csv_files/csv_partitions/2.csv]]}, projection=[int_col, string_col, bigint_col, partition_col], file_type=csv, has_header=false # ensure that correct quote character is used when writing to csv diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 53ca8d81b9e4..95b9b5a9252e 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -72,10 +72,10 @@ WITH RECURSIVE nodes AS ( # trivial recursive CTE works query I rowsort -WITH RECURSIVE nodes AS ( +WITH RECURSIVE nodes AS ( SELECT 1 as id - UNION ALL - SELECT id + 1 as id + UNION ALL + SELECT id + 1 as id FROM nodes WHERE id < 10 ) @@ -94,10 +94,10 @@ SELECT * FROM nodes # explain trivial recursive CTE query TT -EXPLAIN WITH RECURSIVE nodes AS ( +EXPLAIN WITH RECURSIVE nodes AS ( SELECT 1 as id - UNION ALL - SELECT id + 1 as id + UNION ALL + SELECT id + 1 as id FROM nodes WHERE id < 10 ) @@ -140,7 +140,7 @@ set datafusion.execution.batch_size = 2; query TT EXPLAIN WITH RECURSIVE balances AS ( SELECT * from balance - UNION ALL + UNION ALL SELECT time + 1 as time, name, account_balance + 10 as account_balance FROM balances WHERE time < 10 @@ -160,7 +160,7 @@ logical_plan physical_plan 01)SortExec: expr=[time@0 ASC NULLS LAST, name@1 ASC NULLS LAST, account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--RecursiveQueryExec: name=balances, is_distinct=false -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], file_type=csv, has_header=true 04)----CoalescePartitionsExec 05)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] 06)--------CoalesceBatchesExec: target_batch_size=2 @@ -174,7 +174,7 @@ physical_plan query ITI WITH RECURSIVE balances AS ( SELECT * from balance - UNION ALL + UNION ALL SELECT time + 1 as time, name, account_balance + 10 as account_balance FROM balances WHERE time < 10 @@ -230,7 +230,7 @@ query ITI WITH RECURSIVE balances AS ( SELECT time as time, name as name, account_balance as account_balance FROM balance - UNION ALL + UNION ALL SELECT time + 1 as time, balances.name, account_balance + growth.account_growth as account_balance FROM balances JOIN growth @@ -281,10 +281,10 @@ ORDER BY time, name, account_balance # recursive CTE with aggregations works query I rowsort -WITH RECURSIVE nodes AS ( +WITH RECURSIVE nodes AS ( SELECT 1 as id - UNION ALL - SELECT id + 1 as id + UNION ALL + SELECT id + 1 as id FROM nodes WHERE id < 10 ) @@ -299,10 +299,10 @@ CREATE TABLE t(a BIGINT) AS VALUES(1),(2),(3); # referencing CTE multiple times does not error query II rowsort WITH RECURSIVE my_cte AS ( - SELECT a from t - UNION ALL + SELECT a from t + UNION ALL SELECT a+2 as a - FROM my_cte + FROM my_cte WHERE a<5 ) SELECT * FROM my_cte t1, my_cte @@ -619,7 +619,7 @@ WITH RECURSIVE region_sales AS ( SELECT sp.manager_id AS salesperson_id, SUM(rs.amount) AS amount, - MIN(rs.level) + 1 as level + MIN(rs.level) + 1 as level FROM region_sales rs INNER JOIN salespersons sp ON rs.salesperson_id = sp.salesperson_id @@ -630,7 +630,7 @@ WITH RECURSIVE region_sales AS ( SELECT salesperson_id, MAX(amount) as amount, - MAX(level) as hierarchy_level + MAX(level) as hierarchy_level FROM region_sales GROUP BY @@ -851,9 +851,9 @@ query error DataFusion error: Error during planning: table 'datafusion\.public\. # Test duplicate CTE names in different subqueries in the FROM clause. query III rowsort -SELECT * FROM - (WITH t AS (select 400 as e) SELECT * FROM t) t1, - (WITH t AS (select 500 as e) SELECT * FROM t) t2, +SELECT * FROM + (WITH t AS (select 400 as e) SELECT * FROM t) t1, + (WITH t AS (select 500 as e) SELECT * FROM t) t2, t ---- 400 500 1 diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index 0798c8cc1453..5e229075273d 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -771,7 +771,7 @@ query TT explain select c1 from t; ---- logical_plan TableScan: t projection=[c1] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/empty.csv]]}, projection=[c1], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/empty.csv]]}, projection=[c1], file_type=csv, has_header=true statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/dictionary.slt b/datafusion/sqllogictest/test_files/dictionary.slt index b6923fcc944d..778b3537d1bf 100644 --- a/datafusion/sqllogictest/test_files/dictionary.slt +++ b/datafusion/sqllogictest/test_files/dictionary.slt @@ -412,7 +412,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # try literal = col to verify order doesn't matter # filter should not cast column2 @@ -425,7 +425,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Now query using an integer which must be coerced into a dictionary string @@ -443,7 +443,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column2@1 = 1 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Window Functions query I diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 5aa7c89c15d5..b4a491619e89 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -102,7 +102,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[first_value(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], first_value(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true # ON expressions are not a sub-set of the ORDER BY expressions query error SELECT DISTINCT ON expressions must match initial ORDER BY expressions diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 0fb64587a9e6..037565ce05f9 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -46,7 +46,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: c2@1 > 10, projection=[c1@0] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], file_type=csv, has_header=true # explain_csv_exec_scan_config @@ -77,7 +77,7 @@ explain SELECT c1 FROM aggregate_test_100_with_order order by c1 ASC limit 10 logical_plan 01)Sort: aggregate_test_100_with_order.c1 ASC NULLS LAST, fetch=10 02)--TableScan: aggregate_test_100_with_order projection=[c1] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_test_100_order_by_c1_asc.csv]]}, projection=[c1], limit=10, output_ordering=[c1@0 ASC NULLS LAST], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_test_100_order_by_c1_asc.csv]]}, projection=[c1], limit=10, output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true ## explain_physical_plan_only @@ -130,7 +130,7 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test ---- logical_plan TableScan: simple_explain_test projection=[a, b, c] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true # create a sink table, path is same with aggregate_test_100 table # we do not overwrite this file, we only assert plan. @@ -165,7 +165,7 @@ logical_plan physical_plan 01)DataSinkExec: sink=StreamWrite { location: "../../testing/data/csv/aggregate_test_100.csv", batch_size: 8192, encoding: Csv, header: true, .. } 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true # test EXPLAIN VERBOSE query TT @@ -231,12 +231,12 @@ logical_plan after common_sub_expression_eliminate SAME TEXT AS ABOVE logical_plan after eliminate_group_by_constant SAME TEXT AS ABOVE logical_plan after optimize_projections SAME TEXT AS ABOVE logical_plan TableScan: simple_explain_test projection=[a, b, c] -initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true -initial_physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] -initial_physical_plan_with_schema CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +initial_physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true +initial_physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +initial_physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] physical_plan after OutputRequirements 01)OutputRequirementExec -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -246,14 +246,14 @@ physical_plan after EnforceSorting SAME TEXT AS ABOVE physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE -physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +physical_plan after OutputRequirements DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after LimitPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true -physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] -physical_plan_with_schema CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true +physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] ### tests for EXPLAIN with display schema enabled @@ -265,7 +265,7 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test; ---- logical_plan TableScan: simple_explain_test projection=[a, b, c] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], file_type=csv, has_header=true, schema=[a:Int32;N, b:Int32;N, c:Int32;N] statement ok @@ -283,7 +283,7 @@ set datafusion.explain.physical_plan_only = true; query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, file_type=csv, has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] # Parquet scan with statistics collected statement ok @@ -295,7 +295,7 @@ CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-t query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- -physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] # explain verbose with both collect & show statistics on query TT @@ -303,14 +303,14 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ---- initial_physical_plan 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] initial_physical_plan_with_schema 01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] physical_plan after OutputRequirements 01)OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -03)----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -322,13 +322,13 @@ physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after LimitPushdown ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok @@ -340,17 +340,17 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ---- initial_physical_plan 01)GlobalLimitExec: skip=0, fetch=10 -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet initial_physical_plan_with_stats 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] initial_physical_plan_with_schema 01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] physical_plan after OutputRequirements 01)OutputRequirementExec 02)--GlobalLimitExec: skip=0, fetch=10 -03)----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -362,14 +362,14 @@ physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10 -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after LimitPushdown ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 -physical_plan_with_stats ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok @@ -431,7 +431,7 @@ logical_plan 06)--------TableScan: t2 projection=[] physical_plan 01)NestedLoopJoinExec: join_type=LeftSemi -02)--MemoryExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--ProjectionExec: expr=[] 04)----PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index b0c306635dd1..a0264c43622f 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -2022,7 +2022,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[min(t.a) FILTER (WHERE t.a > Int64(1))@0 as x] 02)--AggregateExec: mode=Single, gby=[], aggr=[min(t.a) FILTER (WHERE t.a > Int64(1))] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 66dd99d2ef63..2b3ebcda1520 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2024,10 +2024,10 @@ physical_plan 09)----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] -12)----------------------MemoryExec: partitions=1, partition_sizes=[3] -13)----------------------MemoryExec: partitions=1, partition_sizes=[3] +12)----------------------DataSourceExec: partitions=1, partition_sizes=[3] +13)----------------------DataSourceExec: partitions=1, partition_sizes=[3] -# Columns in the table are a,b,c,d. Source is CsvExec which is ordered by +# Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. # Column c has cardinality 100 (unique entries). Column d has cardinality 5. statement ok @@ -2070,7 +2070,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 as a] 02)--SortExec: expr=[b@1 DESC], preserve_partitioning=[false] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true # Final plan shouldn't have SortExec c ASC, # because table already satisfies this ordering. @@ -2081,7 +2081,7 @@ logical_plan 01)Projection: multiple_ordered_table.a 02)--Sort: multiple_ordered_table.c ASC NULLS LAST 03)----TableScan: multiple_ordered_table projection=[a, c] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # Final plan shouldn't have SortExec a ASC, b ASC, # because table already satisfies this ordering. @@ -2092,7 +2092,7 @@ logical_plan 01)Projection: multiple_ordered_table.a 02)--Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST 03)----TableScan: multiple_ordered_table projection=[a, b] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # test_window_agg_sort statement ok @@ -2276,7 +2276,7 @@ logical_plan 02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d] physical_plan 01)SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], preserve_partitioning=[false] -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], file_type=csv, has_header=true query TT EXPLAIN SELECT a, b, ARRAY_AGG(d ORDER BY d) @@ -2461,7 +2461,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T? @@ -2491,7 +2491,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, sum(s.amount)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)] 03)----SortExec: expr=[amount@1 DESC], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2536,7 +2536,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, sum(s.amount)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2574,7 +2574,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, sum(s.amount)@3 as sum1] 02)--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=PartiallySorted([0]) 03)----SortExec: TopK(fetch=10), expr=[country@1 ASC NULLS LAST, amount@2 DESC], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query TI?R rowsort SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2611,7 +2611,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, sum(s.amount)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(s.amount) ORDER BY [s.country DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2647,7 +2647,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, sum(s.amount)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted 03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2680,7 +2680,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 03)----SortExec: expr=[amount@1 DESC], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -2711,7 +2711,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, @@ -2743,7 +2743,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], first_value(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query TRR? SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, @@ -2773,7 +2773,7 @@ physical_plan 01)ProjectionExec: expr=[country@0 as country, sum(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[sum(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@2 ASC NULLS LAST], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query TR? SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, @@ -2806,7 +2806,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, sum(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], sum(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2839,7 +2839,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, sum(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], sum(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2878,8 +2878,8 @@ physical_plan 04)------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] -07)------------MemoryExec: partitions=1, partition_sizes=[1] -08)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)------------DataSourceExec: partitions=1, partition_sizes=[1] query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2924,7 +2924,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2960,7 +2960,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query TRR @@ -2996,7 +2996,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3022,7 +3022,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3047,7 +3047,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 @@ -3071,7 +3071,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------SortExec: expr=[ts@0 DESC], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 @@ -3095,7 +3095,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 05)--------SortExec: expr=[amount@0 ASC NULLS LAST], preserve_partitioning=[true] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3125,7 +3125,7 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 08)--------------SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[true] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query T? SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3161,7 +3161,7 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 08)--------------SortExec: expr=[amount@1 DESC], preserve_partitioning=[true] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -3361,7 +3361,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query IRI SELECT s.sn, s.amount, 2*s.sn @@ -3430,9 +3430,9 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[sum(l.amount)] 08)--------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1, projection=[amount@1, sn@2, amount@3] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -11)------------------MemoryExec: partitions=1, partition_sizes=[1] +11)------------------DataSourceExec: partitions=1, partition_sizes=[1] query IRR SELECT r.sn, SUM(l.amount), r.amount @@ -3579,7 +3579,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 09)----------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] 10)------------------BoundedWindowAggExec: wdw=[sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -11)--------------------MemoryExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query ITIPTRR @@ -3762,7 +3762,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(foo.x)] 04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -05)--------MemoryExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] query I SELECT FIRST_VALUE(x) @@ -3784,7 +3784,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(foo.x)] 04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -05)--------MemoryExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] # Since both ordering requirements are satisfied, there shouldn't be # any SortExec in the final plan. @@ -3805,7 +3805,7 @@ physical_plan 04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[first_value(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], first_value(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], file_type=csv, has_header=true query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, @@ -3839,7 +3839,7 @@ ORDER BY c ASC; logical_plan 01)Sort: multiple_ordered_table.c ASC NULLS LAST 02)--TableScan: multiple_ordered_table projection=[c] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -3873,10 +3873,10 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true 06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] 07)----------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # reset partition number to 8. statement ok @@ -3917,7 +3917,7 @@ physical_plan 04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # drop table multiple_ordered_table_with_pk statement ok @@ -3958,7 +3958,7 @@ physical_plan 04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -3980,7 +3980,7 @@ physical_plan 01)AggregateExec: mode=Single, gby=[c@0 as c, sum1@1 as sum1], aggr=[], ordering_mode=PartiallySorted([0]) 02)--ProjectionExec: expr=[c@0 as c, sum(multiple_ordered_table_with_pk.d)@1 as sum1] 03)----AggregateExec: mode=Single, gby=[c@0 as c], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb @@ -4000,7 +4000,7 @@ physical_plan 02)--WindowAggExec: wdw=[sum(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 03)----ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 04)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4031,10 +4031,10 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] 04)------ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 05)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true 07)------ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 08)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -09)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], has_header=true +09)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4063,10 +4063,10 @@ physical_plan 02)--CrossJoinExec 03)----ProjectionExec: expr=[c@0 as c, sum(multiple_ordered_table_with_pk.d)@1 as sum1] 04)------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true 06)----ProjectionExec: expr=[c@0 as c, sum(multiple_ordered_table_with_pk.d)@1 as sum1] 07)------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -08)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], has_header=true +08)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # we do not generate physical plan for Repartition yet (e.g Distribute By queries). query TT @@ -4105,10 +4105,10 @@ physical_plan 01)UnionExec 02)--ProjectionExec: expr=[c@0 as c, a@1 as a, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 03)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], constraints=[PrimaryKey([3])], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true 05)--ProjectionExec: expr=[c@0 as c, a@1 as a, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 06)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -07)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], constraints=[PrimaryKey([3])], has_header=true +07)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # table scan should be simplified. query TT @@ -4123,7 +4123,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[c@0 as c, a@1 as a, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], constraints=[PrimaryKey([3])], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true # limit should be simplified query TT @@ -4142,7 +4142,7 @@ physical_plan 01)ProjectionExec: expr=[c@0 as c, a@1 as a, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 02)--GlobalLimitExec: skip=0, fetch=5 03)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], constraints=[PrimaryKey([3])], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], constraints=[PrimaryKey([3])], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 8; @@ -4180,7 +4180,7 @@ physical_plan 04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; @@ -4203,7 +4203,7 @@ physical_plan 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] 11)--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] -12)----------------------MemoryExec: partitions=1, partition_sizes=[1] +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok @@ -4291,7 +4291,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], has_header=false +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], file_type=csv, has_header=false query I SELECT extract(month from ts) as months @@ -4407,7 +4407,7 @@ physical_plan 10)------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[alias2, alias3] 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], has_header=true +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], file_type=csv, has_header=true # Use PostgreSQL dialect statement ok @@ -4580,7 +4580,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 06)----------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[max(timestamp_table.t1)], lim=[4] 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 -08)--------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/0.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/2.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/3.csv]]}, projection=[t1, c2], has_header=true +08)--------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/0.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/2.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/3.csv]]}, projection=[t1, c2], file_type=csv, has_header=true # Clean up statement ok @@ -4969,7 +4969,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b, c] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]], ordering_mode=Sorted -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], file_type=csv, has_header=true query II? SELECT a, b, ARRAY_AGG(c ORDER BY c DESC) @@ -5114,8 +5114,8 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[count(keywords_stream.keyword)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] -05)--------MemoryExec: partitions=1, partition_sizes=[1] -06)--------MemoryExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query PI SELECT diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 097284a6b71f..c1050ee03245 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -73,7 +73,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I INSERT INTO table_without_values SELECT @@ -133,7 +133,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true @@ -184,7 +184,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I @@ -219,7 +219,7 @@ logical_plan physical_plan 01)DataSinkExec: sink=MemoryTable (partitions=1) 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query I insert into table_without_values select c1 from aggregate_test_100 order by c1; diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index ee4dde6248db..e3822cd920b0 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -128,7 +128,7 @@ physical_plan 01)DataSinkExec: sink=CsvSink(file_groups=[]) 02)--SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] 03)----ProjectionExec: expr=[column1@0 as a, column2@1 as b] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] query I INSERT INTO ordered_insert_test values (5, 1), (4, 2), (7,7), (7,8), (7,9), (7,10), (3, 3), (2, 4), (1, 5); @@ -362,7 +362,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true query I INSERT INTO table_without_values SELECT @@ -423,7 +423,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], file_type=csv, has_header=true @@ -460,7 +460,7 @@ logical_plan physical_plan 01)DataSinkExec: sink=ParquetSink(file_groups=[]) 02)--SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query I insert into table_without_values select c1 from aggregate_test_100 order by c1; diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 1feacc5ebe53..c88f419a9cb2 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -671,14 +671,14 @@ query TT explain select * from t1 inner join t2 on true; ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)CrossJoinExec -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] 03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)----MemoryExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table IF EXISTS t1; @@ -756,8 +756,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -03)----MemoryExec: partitions=1, partition_sizes=[1] -04)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] # Reset the configs to old values statement ok @@ -828,8 +828,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[emp_id@1 as emp_id, name@2 as name, dept_name@0 as dept_name] 02)--NestedLoopJoinExec: join_type=Right, filter=name@0 = Alice OR name@0 = Bob -03)----MemoryExec: partitions=1, partition_sizes=[1] -04)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -905,7 +905,7 @@ JOIN department AS d ON (e.name = 'Alice' OR e.name = 'Bob'); ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--SubqueryAlias: e 03)----Filter: employees.name = Utf8("Alice") OR employees.name = Utf8("Bob") 04)------TableScan: employees projection=[emp_id, name] @@ -915,8 +915,8 @@ physical_plan 01)CrossJoinExec 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: name@1 = Alice OR name@1 = Bob -04)------MemoryExec: partitions=1, partition_sizes=[1] -05)--MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)--DataSourceExec: partitions=1, partition_sizes=[1] # expect no row for Carol query ITT @@ -968,8 +968,8 @@ physical_plan 05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol -08)--------------MemoryExec: partitions=1, partition_sizes=[1] -09)----------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] +09)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITT SELECT e.emp_id, e.name, d.dept_name @@ -1164,9 +1164,9 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6] -08)--------------MemoryExec: partitions=1, partition_sizes=[0] -09)--------------MemoryExec: partitions=1, partition_sizes=[0] -10)----MemoryExec: partitions=1, partition_sizes=[0] +08)--------------DataSourceExec: partitions=1, partition_sizes=[0] +09)--------------DataSourceExec: partitions=1, partition_sizes=[0] +10)----DataSourceExec: partitions=1, partition_sizes=[0] diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 0bdf223a11b7..a1efc1317b4a 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -57,9 +57,9 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true 05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true +06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # preserve_inner_join query IIII nosort @@ -98,11 +98,11 @@ physical_plan 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true 06)--------CoalesceBatchesExec: target_batch_size=8192 07)----------FilterExec: d@3 = 3 08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +09)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index f16b2dbb2d0d..5d311bc43293 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1344,11 +1344,11 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------CoalesceBatchesExec: target_batch_size=2 09)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)------------MemoryExec: partitions=1, partition_sizes=[1] +11)------------DataSourceExec: partitions=1, partition_sizes=[1] # Join on struct query TT @@ -1366,11 +1366,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([s3@0], 2), input_partitions=2 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----CoalesceBatchesExec: target_batch_size=2 08)------RepartitionExec: partitioning=Hash([s4@0], 2), input_partitions=2 09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------MemoryExec: partitions=1, partition_sizes=[1] +10)----------DataSourceExec: partitions=1, partition_sizes=[1] query ?? select join_t3.s3, join_t4.s4 @@ -1409,11 +1409,11 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1440,11 +1440,11 @@ physical_plan 08)--------------CoalesceBatchesExec: target_batch_size=2 09)----------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)--------------------MemoryExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] 12)--------------CoalesceBatchesExec: target_batch_size=2 13)----------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 14)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)--------------------MemoryExec: partitions=1, partition_sizes=[1] +15)--------------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1508,10 +1508,10 @@ physical_plan 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] 09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------MemoryExec: partitions=1, partition_sizes=[1] +10)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1535,12 +1535,12 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 06)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)------CoalesceBatchesExec: target_batch_size=2 10)--------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 11)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] 12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)--------------MemoryExec: partitions=1, partition_sizes=[1] +13)--------------DataSourceExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1565,10 +1565,10 @@ physical_plan 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] 09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------MemoryExec: partitions=1, partition_sizes=[1] +10)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1592,12 +1592,12 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)------CoalesceBatchesExec: target_batch_size=2 10)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 11)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] 12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)--------------MemoryExec: partitions=1, partition_sizes=[1] +13)--------------DataSourceExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1620,10 +1620,10 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1647,12 +1647,12 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------CoalesceBatchesExec: target_batch_size=2 09)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 10)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1678,9 +1678,9 @@ physical_plan 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------MemoryExec: partitions=1, partition_sizes=[1] +09)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1705,11 +1705,11 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)------CoalesceBatchesExec: target_batch_size=2 10)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)------------MemoryExec: partitions=1, partition_sizes=[1] +12)------------DataSourceExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1730,10 +1730,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1755,12 +1755,12 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----CoalesceBatchesExec: target_batch_size=2 08)------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 09)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)------------MemoryExec: partitions=1, partition_sizes=[1] +11)------------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -2085,11 +2085,11 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)----CoalesceBatchesExec: target_batch_size=2 09)------FilterExec: t1_id@0 > 10 10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)----------MemoryExec: partitions=1, partition_sizes=[1] +11)----------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2124,11 +2124,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------FilterExec: t1_id@0 > 22 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)--CoalesceBatchesExec: target_batch_size=2 08)----FilterExec: t2_id@0 > 11 09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)--------MemoryExec: partitions=1, partition_sizes=[1] +10)--------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2602,11 +2602,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----CoalesceBatchesExec: target_batch_size=2 08)------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)----------MemoryExec: partitions=1, partition_sizes=[1] +10)----------DataSourceExec: partitions=1, partition_sizes=[1] # left_join_using_2 query II @@ -2779,12 +2779,12 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] 08)----CoalesceBatchesExec: target_batch_size=2 09)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)----------MemoryExec: partitions=1, partition_sizes=[1] +11)----------DataSourceExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_date32 inner sort merge join on data type (Date32) query DDRTDDRT rowsort @@ -2811,12 +2811,12 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([CAST(t1.c3 AS Decimal128(10, 2))@4], 2), input_partitions=2 06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))] 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)----SortExec: expr=[c3@2 ASC], preserve_partitioning=[true] 10)------CoalesceBatchesExec: target_batch_size=2 11)--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 12)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------MemoryExec: partitions=1, partition_sizes=[1] +13)------------DataSourceExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_decimal right join on data type (Decimal) query DDRTDDRT rowsort @@ -2873,12 +2873,12 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2914,12 +2914,12 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -2973,10 +2973,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -3009,10 +3009,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -3070,12 +3070,12 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3092,12 +3092,12 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3149,10 +3149,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3166,10 +3166,10 @@ physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----------DataSourceExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3251,11 +3251,11 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 07)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true 09)----CoalesceBatchesExec: target_batch_size=2 10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST 11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +12)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # sort merge join should propagate ordering equivalence of the right side # for right join. Hence final requirement rn1 ASC is already satisfied at @@ -3283,13 +3283,13 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true 07)----CoalesceBatchesExec: target_batch_size=2 08)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST 09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 11)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -12)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +12)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_existing_sort = false; @@ -3328,14 +3328,14 @@ physical_plan 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 08)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 09)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true 11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] 12)--------CoalesceBatchesExec: target_batch_size=2 13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 15)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 16)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -17)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +17)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -3367,10 +3367,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true 04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 05)------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # hash join should propagate ordering equivalence of the right side for RIGHT ANTI join. # Hence final requirement rn1 ASC is already satisfied at the end of HashJoinExec. @@ -3394,10 +3394,10 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, has_header=true 04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 05)------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 @@ -3421,8 +3421,8 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true -06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true +06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true # create a table where there more than one valid ordering # that describes table. @@ -3469,10 +3469,10 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true 06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] 07)----------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +08)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true # run query above in multiple partitions statement ok @@ -3512,11 +3512,11 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=2 11)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true 14)------------------CoalesceBatchesExec: target_batch_size=2 15)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 16)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true +17)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT * @@ -3531,9 +3531,9 @@ logical_plan 05)----TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=a@1 < a@0 -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true 03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # Currently datafusion can pushdown filter conditions with scalar UDF into # cross join. @@ -3550,9 +3550,9 @@ logical_plan 05)----TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=example(CAST(a@0 AS Float64), CAST(a@1 AS Float64)) > 3 -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true 03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true #### # Config teardown @@ -3930,8 +3930,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------MemoryExec: partitions=1, partition_sizes=[1] -06)------MemoryExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3987,8 +3987,8 @@ physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -04)------MemoryExec: partitions=1, partition_sizes=[1] -05)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] +05)------DataSourceExec: partitions=1, partition_sizes=[1] # Null build indices: @@ -4048,8 +4048,8 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=3 03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -05)--------MemoryExec: partitions=1, partition_sizes=[1] -06)------MemoryExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)------DataSourceExec: partitions=1, partition_sizes=[1] # Test CROSS JOIN LATERAL syntax (planning) @@ -4154,10 +4154,10 @@ physical_plan 03)----AggregateExec: mode=Single, gby=[sn@1 as sn, ts@0 as ts, amount@2 as amount, currency@3 as currency], aggr=[last_value(e.rate)] 04)------CoalesceBatchesExec: target_batch_size=3 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] -06)----------MemoryExec: partitions=1, partition_sizes=[0] +06)----------DataSourceExec: partitions=1, partition_sizes=[0] 07)----------CoalesceBatchesExec: target_batch_size=3 08)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] -09)--------------MemoryExec: partitions=1, partition_sizes=[0] +09)--------------DataSourceExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; @@ -4183,9 +4183,9 @@ logical_plan 04)----TableScan: right_table projection=[x, y, z] physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=a@0 < x@1 -02)--MemoryExec: partitions=1, partition_sizes=[0] +02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] -04)----MemoryExec: partitions=1, partition_sizes=[0] +04)----DataSourceExec: partitions=1, partition_sizes=[0] query TT EXPLAIN SELECT * FROM left_table JOIN right_table ON left_table.a= c2@1 -03)----MemoryExec: partitions=1, partition_sizes=[1] -04)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] ## Test !join.on.is_empty() && join.filter.is_some() query TT @@ -4279,8 +4279,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)], filter=c2@0 >= c2@1 -03)----MemoryExec: partitions=1, partition_sizes=[1] -04)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----DataSourceExec: partitions=1, partition_sizes=[1] ## Add more test cases for join limit pushdown statement ok @@ -4343,8 +4343,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], limit=2, has_header=true -04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true ###### ## RIGHT JOIN w/ LIMIT @@ -4377,8 +4377,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], has_header=true -04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], limit=2, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true ###### ## FULL JOIN w/ LIMIT @@ -4414,8 +4414,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(a@0, b@0)] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], has_header=true -04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true statement ok drop table t1; @@ -4454,8 +4454,8 @@ physical_plan 02)--ProjectionExec: expr=[] 03)----CoalesceBatchesExec: target_batch_size=3 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(binary_col@0, binary_col@0)] -05)--------MemoryExec: partitions=1, partition_sizes=[1] -06)--------MemoryExec: partitions=1, partition_sizes=[1] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------DataSourceExec: partitions=1, partition_sizes=[1] # Test hash join sort push down # Issue: https://github.com/apache/datafusion/issues/13559 @@ -4488,12 +4488,12 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=3 07)------------FilterExec: b@1 > 3, projection=[a@0] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] 11)--------CoalesceBatchesExec: target_batch_size=3 12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)--------------MemoryExec: partitions=1, partition_sizes=[1] +14)--------------DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; @@ -4515,12 +4515,12 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=3 07)------------FilterExec: b@1 > 3, projection=[a@0] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] 11)--------CoalesceBatchesExec: target_batch_size=3 12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)--------------MemoryExec: partitions=1, partition_sizes=[1] +14)--------------DataSourceExec: partitions=1, partition_sizes=[1] query III select * from test where a in (select a from test where b > 3) order by c desc nulls first; diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index 0903c2427649..dd310f7f2bf6 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -61,7 +61,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]} +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]}, file_type=json query ? SELECT mycol FROM single_nan @@ -143,4 +143,4 @@ query TT EXPLAIN SELECT id FROM json_partitioned_test WHERE part = 2 ---- logical_plan TableScan: json_partitioned_test projection=[id], full_filters=[json_partitioned_test.part = Int32(2)] -physical_plan JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_json/part=2/data.json]]}, projection=[id] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/partitioned_table_json/part=2/data.json]]}, projection=[id], file_type=json diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index a999149418ef..4e74b27b875f 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -374,7 +374,7 @@ physical_plan 06)----------GlobalLimitExec: skip=6, fetch=3 07)------------CoalesceBatchesExec: target_batch_size=8192, fetch=9 08)--------------FilterExec: a@0 > 3 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); @@ -404,7 +404,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] -06)----------MemoryExec: partitions=1 +06)----------DataSourceExec: partitions=1 statement ok set datafusion.explain.show_sizes = true; @@ -636,7 +636,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true # Applying offset & limit when multiple streams from union # the plan must still have a global limit to apply the offset @@ -662,11 +662,11 @@ physical_plan 04)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] 05)--------ProjectionExec: expr=[CAST(c@0 AS Int64) as c] 06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true 08)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] 09)--------ProjectionExec: expr=[CAST(d@0 AS Int64) as c] 10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[d], has_header=true +11)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[d], file_type=csv, has_header=true # Applying LIMIT & OFFSET to subquery. query III @@ -739,10 +739,8 @@ physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--GlobalLimitExec: skip=0, fetch=10 03)----CrossJoinExec -04)------GlobalLimitExec: skip=0, fetch=1 -05)--------MemoryExec: partitions=1, partition_sizes=[1] -06)------GlobalLimitExec: skip=0, fetch=10 -07)--------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1], fetch=10 query IIII @@ -765,10 +763,8 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=2 02)--CrossJoinExec -03)----GlobalLimitExec: skip=0, fetch=2 -04)------MemoryExec: partitions=1, partition_sizes=[1] -05)----GlobalLimitExec: skip=0, fetch=2 -06)------MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 +04)----DataSourceExec: partitions=1, partition_sizes=[1], fetch=2 statement ok drop table testSubQueryLimit; @@ -853,7 +849,7 @@ physical_plan 02)--SortExec: TopK(fetch=1000), expr=[part_key@1 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[1 as foo, part_key@0 as part_key] 04)------CoalescePartitionsExec: fetch=1 -05)--------ParquetExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], limit=1 +05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], limit=1, file_type=parquet query I with selection as ( diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 7ebc80025301..29ef506aa070 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -115,7 +115,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int_field@0 > 0 -03)----MemoryExec: partitions=1, partition_sizes=[0] +03)----DataSourceExec: partitions=1, partition_sizes=[0] statement ok drop table table_with_map; diff --git a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt index abf48fac5364..e8700b1fea27 100644 --- a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt +++ b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt @@ -47,7 +47,7 @@ physical_plan 01)SortPreservingMergeExec: [a_big@0 ASC NULLS LAST, b@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN @@ -63,7 +63,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true # Cast to larger types as well as preserving ordering # doesn't invalidate lexicographical ordering. @@ -84,7 +84,7 @@ physical_plan 01)SortPreservingMergeExec: [a_big@1 ASC NULLS LAST, b@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true # test for common rename query TT @@ -99,7 +99,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN @@ -113,7 +113,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, a@0 as a_big, b@1 as b] -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true # test for cast Utf8 @@ -136,7 +136,7 @@ physical_plan 02)--SortExec: expr=[a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(a@0 AS Utf8) as a_str, b@1 as b] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true # We cannot determine a+b is ordered from the # invariant [a ASC, b ASC] is satisfied. Hence @@ -152,7 +152,7 @@ logical_plan 02)--TableScan: multiple_ordered_table projection=[a, b] physical_plan 01)SortExec: expr=[a@0 + b@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true # With similar reasoning above. It is not guaranteed sum_expr is ordered # Hence we should see a SortExec with sum_expr ASC in the plan. @@ -171,4 +171,4 @@ physical_plan 02)--SortExec: expr=[sum_expr@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(a@0 + b@1 AS Int64) as sum_expr, a@0 as a, b@1 as b] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/operator.slt b/datafusion/sqllogictest/test_files/operator.slt index 83bd1d7ee3cd..8fd0a7a61033 100644 --- a/datafusion/sqllogictest/test_files/operator.slt +++ b/datafusion/sqllogictest/test_files/operator.slt @@ -280,7 +280,7 @@ WHERE int64 < 5 AND uint64 < 5 AND float64 < 5 AND decimal < 5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 < 5 AND uint64@7 < 5 AND float64@9 < 5 AND decimal@10 < Some(500),5,2 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## < negative integer (expect no casts) query TT @@ -290,7 +290,7 @@ WHERE int64 < -5 AND uint64 < -5 AND float64 < -5 AND decimal < -5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 < -5 AND CAST(uint64@7 AS Decimal128(20, 0)) < Some(-5),20,0 AND float64@9 < -5 AND decimal@10 < Some(-500),5,2 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## < decimal (expect casts for integers to float) query TT @@ -300,7 +300,7 @@ WHERE int64 < 5.1 AND uint64 < 5.1 AND float64 < 5.1 AND decimal < 5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) < 5.1 AND CAST(uint64@7 AS Float64) < 5.1 AND float64@9 < 5.1 AND decimal@10 < Some(510),5,2 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## < negative decimal (expect casts for integers to float) query TT @@ -310,7 +310,7 @@ WHERE int64 < -5.1 AND uint64 < -5.1 AND float64 < -5.1 AND decimal < -5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) < -5.1 AND CAST(uint64@7 AS Float64) < -5.1 AND float64@9 < -5.1 AND decimal@10 < Some(-510),5,2 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ############### Equality ############### @@ -323,7 +323,7 @@ WHERE int64 = 5 AND uint64 = 5 AND float64 = 5 AND decimal = 5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 = 5 AND uint64@7 = 5 AND float64@9 = 5 AND decimal@10 = Some(500),5,2 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## = negative integer (expect no casts) query TT @@ -333,7 +333,7 @@ WHERE int64 = -5 AND uint64 = -5 AND float64 = -5 AND decimal = -5; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: int64@3 = -5 AND CAST(uint64@7 AS Decimal128(20, 0)) = Some(-5),20,0 AND float64@9 = -5 AND decimal@10 = Some(-500),5,2 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## = decimal (expect casts for integers to float) query TT @@ -343,7 +343,7 @@ WHERE int64 = 5.1 AND uint64 = 5.1 AND float64 = 5.1 AND decimal = 5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) = 5.1 AND CAST(uint64@7 AS Float64) = 5.1 AND float64@9 = 5.1 AND decimal@10 = Some(510),5,2 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## = negative decimal (expect casts for integers to float) query TT @@ -353,7 +353,7 @@ WHERE int64 = -5.1 AND uint64 = -5.1 AND float64 = -5.1 AND decimal = -5.1; physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(int64@3 AS Float64) = -5.1 AND CAST(uint64@7 AS Float64) = -5.1 AND float64@9 = -5.1 AND decimal@10 = Some(-510),5,2 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/options.slt b/datafusion/sqllogictest/test_files/options.slt index 14b0e2d43c63..71ff12e8cc50 100644 --- a/datafusion/sqllogictest/test_files/options.slt +++ b/datafusion/sqllogictest/test_files/options.slt @@ -33,7 +33,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: c0@0 < 1 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] ## # test_disable_coalesce @@ -51,7 +51,7 @@ logical_plan 02)--TableScan: a projection=[c0] physical_plan 01)FilterExec: c0@0 < 1 -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.execution.coalesce_batches = true @@ -74,7 +74,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=1234 02)--FilterExec: c0@0 < 1 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 0f8417169725..d95001b81b08 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -166,7 +166,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] 02)--SortExec: expr=[c2@1 ASC NULLS LAST, c3@2 ASC NULLS LAST], preserve_partitioning=[false] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true query II SELECT c2, c3 FROM aggregate_test_100 ORDER BY c2, c3, c2 @@ -457,7 +457,7 @@ physical_plan 01)SortPreservingMergeExec: [result@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[b@1 + a@0 + c@2 as result] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST], [b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST], [b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], file_type=csv, has_header=true statement ok drop table multiple_ordered_table; @@ -488,7 +488,7 @@ physical_plan 01)SortPreservingMergeExec: [db15@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0, 1659537600000000000) as db15] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], has_header=false +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false query TT EXPLAIN SELECT DATE_TRUNC('DAY', ts) as dt_day @@ -503,7 +503,7 @@ physical_plan 01)SortPreservingMergeExec: [dt_day@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[date_trunc(DAY, ts@0) as dt_day] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], has_header=false +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=false statement ok drop table csv_with_timestamps; @@ -546,7 +546,7 @@ physical_plan 01)SortPreservingMergeExec: [atan_c11@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[atan(c11@0) as atan_c11] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT CEIL(c11) as ceil_c11 @@ -561,7 +561,7 @@ physical_plan 01)SortPreservingMergeExec: [ceil_c11@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[ceil(c11@0) as ceil_c11] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT LOG(c12, c11) as log_c11_base_c12 @@ -576,7 +576,7 @@ physical_plan 01)SortPreservingMergeExec: [log_c11_base_c12@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[log(c12@1, CAST(c11@0 AS Float64)) as log_c11_base_c12] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC NULLS LAST]], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC NULLS LAST]], file_type=csv, has_header=true query TT EXPLAIN SELECT LOG(c11, c12) as log_c12_base_c11 @@ -591,7 +591,7 @@ physical_plan 01)SortPreservingMergeExec: [log_c12_base_c11@0 DESC NULLS LAST] 02)--ProjectionExec: expr=[log(CAST(c11@0 AS Float64), c12@1) as log_c12_base_c11] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC NULLS LAST]], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC NULLS LAST]], file_type=csv, has_header=true statement ok drop table aggregate_test_100; @@ -653,7 +653,7 @@ EXPLAIN SELECT o_orderkey, o_orderstatus FROM orders ORDER BY o_orderkey ASC logical_plan 01)Sort: orders.o_orderkey ASC NULLS LAST 02)--TableScan: orders projection=[o_orderkey, o_orderstatus] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/orders.csv]]}, projection=[o_orderkey, o_orderstatus], output_ordering=[o_orderkey@0 ASC NULLS LAST], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/orders.csv]]}, projection=[o_orderkey, o_orderstatus], output_ordering=[o_orderkey@0 ASC NULLS LAST], file_type=csv, has_header=true # Create external table with DDL ordered columns that are missing @@ -692,7 +692,7 @@ logical_plan 02)--TableScan: t1 projection=[id, name] physical_plan 01)SortExec: expr=[id@0 DESC, name@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] query IT SELECT * FROM t1 ORDER BY id ASC, id, name, id DESC; @@ -711,7 +711,7 @@ logical_plan 02)--TableScan: t1 projection=[id, name] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST, name@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # Minimal reproduction of issue 5970 @@ -786,7 +786,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] 10)------------------ProjectionExec: expr=[column1@0 as t] -11)--------------------MemoryExec: partitions=1, partition_sizes=[1] +11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] 12)------ProjectionExec: expr=[1 as m, t@0 as t] 13)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] 14)----------CoalesceBatchesExec: target_batch_size=8192 @@ -794,7 +794,7 @@ physical_plan 16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 17)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] 18)------------------ProjectionExec: expr=[column1@0 as t] -19)--------------------MemoryExec: partitions=1, partition_sizes=[1] +19)--------------------DataSourceExec: partitions=1, partition_sizes=[1] ##### # Multi column sorting with lists @@ -1025,7 +1025,7 @@ physical_plan 02)--SortExec: TopK(fetch=5), expr=[c_str@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(c@0 AS Utf8) as c_str] 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true # Casting from numeric to numeric types preserves the ordering @@ -1055,7 +1055,7 @@ physical_plan 01)SortPreservingMergeExec: [c_bigint@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[CAST(c@0 AS Int64) as c_bigint] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table ordered_table; @@ -1091,7 +1091,7 @@ physical_plan 02)--SortExec: TopK(fetch=5), expr=[abs_c@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[abs(c@0) as abs_c] 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table ordered_table; @@ -1125,7 +1125,7 @@ physical_plan 01)SortPreservingMergeExec: [abs_c@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[abs(c@0) as abs_c] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true # Boolean to integer casts preserve the order. statement ok @@ -1151,7 +1151,7 @@ physical_plan 01)SortPreservingMergeExec: [c@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(inc_col@0 > desc_col@1 AS Int32) as c] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], file_type=csv, has_header=true # Union a query with the actual data and one with a constant query I @@ -1175,7 +1175,7 @@ physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true statement ok set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = true; @@ -1194,7 +1194,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true statement ok set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = false; @@ -1214,7 +1214,7 @@ physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] 02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], file_type=csv, has_header=true # Test: inputs into union with different orderings @@ -1241,10 +1241,10 @@ physical_plan 03)----UnionExec 04)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a@2 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] 05)--------ProjectionExec: expr=[b@1 as b, c@2 as c, a@0 as a, NULL as a0, d@3 as d] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[c@2 ASC NULLS LAST], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true 07)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] 08)--------ProjectionExec: expr=[b@1 as b, c@2 as c, NULL as a, a0@0 as a0, d@3 as d] -09)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, b, c, d], output_ordering=[c@2 ASC NULLS LAST], has_header=true +09)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, b, c, d], output_ordering=[c@2 ASC NULLS LAST], file_type=csv, has_header=true # Test: run the query from above query IIII @@ -1327,7 +1327,7 @@ EXPLAIN SELECT c1, c2 FROM table_with_ordered_pk ORDER BY c1, c2; logical_plan 01)Sort: table_with_ordered_pk.c1 ASC NULLS LAST, table_with_ordered_pk.c2 ASC NULLS LAST 02)--TableScan: table_with_ordered_pk projection=[c1, c2] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], constraints=[PrimaryKey([0])], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], constraints=[PrimaryKey([0])], file_type=csv, has_header=true statement ok drop table table_with_ordered_pk; @@ -1356,7 +1356,7 @@ EXPLAIN SELECT c1, SUM(c2) as sum_c2 FROM table_with_ordered_not_null GROUP BY c physical_plan 01)ProjectionExec: expr=[c1@0 as c1, sum(table_with_ordered_not_null.c2)@1 as sum_c2] 02)--AggregateExec: mode=Single, gby=[c1@0 as c1], aggr=[sum(table_with_ordered_not_null.c2)], ordering_mode=Sorted -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table table_with_ordered_not_null; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 28f1a6a3780d..66bf44e05856 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -79,7 +79,7 @@ SELECT * FROM test_table ORDER BY int_col; 5 eee 500 1970-01-06 6 fff 600 1970-01-07 -# Check output plan, expect no "output_ordering" clause in the physical_plan -> ParquetExec: +# Check output plan, expect no "output_ordering" clause in the physical_plan -> DataSourceExec: query TT EXPLAIN SELECT int_col, string_col FROM test_table @@ -91,7 +91,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] 02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], file_type=parquet # Tear down test_table: statement ok @@ -109,7 +109,7 @@ STORED AS PARQUET WITH ORDER (string_col ASC NULLS LAST, int_col ASC NULLS LAST) LOCATION 'test_files/scratch/parquet/test_table'; -# Check output plan, expect an "output_ordering" clause in the physical_plan -> ParquetExec: +# Check output plan, expect an "output_ordering" clause in the physical_plan -> DataSourceExec: query TT EXPLAIN SELECT int_col, string_col FROM test_table @@ -120,7 +120,7 @@ logical_plan 02)--TableScan: test_table projection=[int_col, string_col] physical_plan 01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] -02)--ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], file_type=parquet # Add another file to the directory underlying test_table query I @@ -130,7 +130,7 @@ STORED AS PARQUET; ---- 3 -# Check output plan again, expect no "output_ordering" clause in the physical_plan -> ParquetExec, +# Check output plan again, expect no "output_ordering" clause in the physical_plan -> DataSourceExec, # due to there being more files than partitions: query TT EXPLAIN SELECT int_col, string_col @@ -143,7 +143,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] 02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col], file_type=parquet # Perform queries using MIN and MAX @@ -411,7 +411,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% statement ok @@ -459,7 +459,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -510,7 +510,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -625,7 +625,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 LIKE f% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok drop table foo diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 806886b07170..0ff0ba24e472 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -63,9 +63,9 @@ LOCATION 'test_files/scratch/parquet_filter_pushdown/parquet_table/'; statement ok set datafusion.execution.parquet.pushdown_filters = false; -# When filter pushdown is not enabled, ParquetExec only filters based on +# When filter pushdown is not enabled, DataSourceExec only filters based on # metadata, so a FilterExec is required to filter the -# output of the `ParquetExec` +# output of the `DataSourceExec` query T select a from t where b > 2 ORDER BY a; @@ -85,10 +85,10 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], predicate=b@1 > 2, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2, required_guarantees=[] -# When filter pushdown *is* enabled, ParquetExec can filter exactly, +# When filter pushdown *is* enabled, DataSourceExec can filter exactly, # not just metadata, so we expect to see no FilterExec query T select a from t_pushdown where b > 2 ORDER BY a; @@ -113,7 +113,7 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: b@1 > 2, projection=[a@0] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2, required_guarantees=[] +06)----------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2, required_guarantees=[] # also test querying on columns that are not in all the files query T @@ -131,7 +131,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2 AND a_null_count@4 != a_row_count@3, required_guarantees=[] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2 AND a_null_count@4 != a_row_count@3, required_guarantees=[] query I @@ -148,7 +148,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[b], predicate=a@0 = bar, pruning_predicate=a_null_count@2 != a_row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != a_row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] ## cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt index b68d4f52d21c..d325ca423dac 100644 --- a/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_sorted_statistics.slt @@ -109,7 +109,7 @@ ORDER BY int_col, bigint_col; logical_plan 01)Sort: test_table.int_col ASC NULLS LAST, test_table.bigint_col ASC NULLS LAST 02)--TableScan: test_table projection=[int_col, bigint_col] -physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[int_col, bigint_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[int_col, bigint_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Another planning test, but project on a column with unsupported statistics # We should be able to ignore this and look at only the relevant statistics @@ -123,9 +123,7 @@ logical_plan 02)--Sort: test_table.int_col ASC NULLS LAST, test_table.bigint_col ASC NULLS LAST 03)----Projection: test_table.string_col, test_table.int_col, test_table.bigint_col 04)------TableScan: test_table projection=[int_col, string_col, bigint_col] -physical_plan -01)ProjectionExec: expr=[string_col@1 as string_col] -02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[int_col, string_col, bigint_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[string_col], file_type=parquet # Clean up & recreate but sort on descending column statement ok @@ -157,7 +155,7 @@ ORDER BY descending_col DESC NULLS LAST, bigint_col ASC NULLS LAST; logical_plan 01)Sort: test_table.descending_col DESC NULLS LAST, test_table.bigint_col ASC NULLS LAST 02)--TableScan: test_table projection=[descending_col, bigint_col] -physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet]]}, projection=[descending_col, bigint_col], output_ordering=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet]]}, projection=[descending_col, bigint_col], output_ordering=[descending_col@0 DESC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Clean up & re-create with partition columns in sort order statement ok @@ -191,7 +189,7 @@ ORDER BY partition_col, int_col, bigint_col; logical_plan 01)Sort: test_table.partition_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST, test_table.bigint_col ASC NULLS LAST 02)--TableScan: test_table projection=[int_col, bigint_col, partition_col] -physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[int_col, bigint_col, partition_col], output_ordering=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[int_col, bigint_col, partition_col], output_ordering=[partition_col@2 ASC NULLS LAST, int_col@0 ASC NULLS LAST, bigint_col@1 ASC NULLS LAST], file_type=parquet # Clean up & re-create with overlapping column in sort order # This will test the ability to sort files with overlapping statistics @@ -224,7 +222,7 @@ logical_plan 02)--TableScan: test_table projection=[int_col, bigint_col, overlapping_col] physical_plan 01)SortPreservingMergeExec: [overlapping_col@2 ASC NULLS LAST] -02)--ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, overlapping_col], output_ordering=[overlapping_col@2 ASC NULLS LAST] +02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet]]}, projection=[int_col, bigint_col, overlapping_col], output_ordering=[overlapping_col@2 ASC NULLS LAST], file_type=parquet # Clean up & re-create with constant column in sort order # This will require a sort because the # of required file groups (3) @@ -259,4 +257,4 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [constant_col@0 ASC NULLS LAST] 02)--SortExec: expr=[constant_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col] +03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=A/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=B/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_sorted_statistics/test_table/partition_col=C/2.parquet]]}, projection=[constant_col], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index d03d9a36aa8e..8bf1caa00370 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -675,13 +675,13 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------FilterExec: l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=true 09)----CoalesceBatchesExec: target_batch_size=8192 10)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 11)--------CoalesceBatchesExec: target_batch_size=8192 12)----------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true +14)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], file_type=csv, has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -767,16 +767,16 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true 10)----------CoalesceBatchesExec: target_batch_size=8192 11)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 12)--------------CoalesceBatchesExec: target_batch_size=8192 13)----------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] 14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true +15)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true 16)------CoalesceBatchesExec: target_batch_size=8192 17)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -18)----------MemoryExec: partitions=1, partition_sizes=[1] +18)----------DataSourceExec: partitions=1, partition_sizes=[1] # Inlist simplification @@ -793,7 +793,7 @@ query TT explain select x from t where x NOT IN (1,2,3,4) OR x NOT IN (5,6,7,8); ---- logical_plan TableScan: t projection=[x] -physical_plan MemoryExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select x from t where x IN (1,2,3,4,5) AND x NOT IN (1,2,3,4); @@ -804,7 +804,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: x@0 = 5 -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query TT explain select x from t where x NOT IN (1,2,3,4,5) AND x IN (1,2,3); diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index b5bcb5b4c6f7..0f0cbac1fa32 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -183,7 +183,7 @@ query TT explain select b from memory_table; ---- logical_plan TableScan: memory_table projection=[b] -physical_plan MemoryExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query I select b from memory_table; @@ -248,7 +248,7 @@ logical_plan 02)--TableScan: t projection=[column1] physical_plan 01)ProjectionExec: expr=[get_field(column1@0, c0) as t.column1[c0]] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/references.slt b/datafusion/sqllogictest/test_files/references.slt index 4c3ac68aebd1..0e72c5e5a29e 100644 --- a/datafusion/sqllogictest/test_files/references.slt +++ b/datafusion/sqllogictest/test_files/references.slt @@ -107,7 +107,7 @@ logical_plan physical_plan 01)SortExec: expr=[....@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[....@0 as ...., ....@0 as c3] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Partial qualified name diff --git a/datafusion/sqllogictest/test_files/regexp.slt b/datafusion/sqllogictest/test_files/regexp.slt index 80f94e21d1fe..ce39434e6827 100644 --- a/datafusion/sqllogictest/test_files/regexp.slt +++ b/datafusion/sqllogictest/test_files/regexp.slt @@ -872,7 +872,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 LIKE %oo% -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # Ensure casting / coercion works for all operators # (there should be no casts to Utf8) @@ -889,7 +889,7 @@ logical_plan 02)--TableScan: dict_table projection=[column1] physical_plan 01)ProjectionExec: expr=[column1@0 LIKE %oo% as dict_table.column1 LIKE Utf8("%oo%"), column1@0 NOT LIKE %oo% as dict_table.column1 NOT LIKE Utf8("%oo%"), column1@0 ILIKE %oo% as dict_table.column1 ILIKE Utf8("%oo%"), column1@0 NOT ILIKE %oo% as dict_table.column1 NOT ILIKE Utf8("%oo%")] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table strings diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 36a326928fad..70666346e2ca 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -48,7 +48,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # disable round robin repartitioning statement ok @@ -65,7 +65,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] -05)--------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # Cleanup diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 9ba96e985fe5..2d03a53ec2ad 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -61,7 +61,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -77,7 +77,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -102,7 +102,7 @@ physical_plan 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: column1@0 != 42 -05)--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:272..538, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..278], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:278..547]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +05)--------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:272..538, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..278], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:278..547]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered @@ -138,7 +138,7 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: column1@0 != 42 -04)------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..269], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..273], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:273..547], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:269..538]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..269], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..273], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:273..547], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:269..538]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # Cleanup statement ok @@ -185,7 +185,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:5..10], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:10..15], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:15..18]]}, projection=[column1], has_header=true +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:5..10], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:10..15], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/csv_table/1.csv:15..18]]}, projection=[column1], file_type=csv, has_header=true # Cleanup statement ok @@ -228,7 +228,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----JsonExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:0..18], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:18..36], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:36..54], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:54..70]]}, projection=[column1] +03)----DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:0..18], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:18..36], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:36..54], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/json_table/1.json:54..70]]}, projection=[column1], file_type=json # Cleanup statement ok @@ -253,7 +253,7 @@ query TT EXPLAIN SELECT * FROM arrow_table ---- logical_plan TableScan: arrow_table projection=[f0, f1, f2] -physical_plan ArrowExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:0..461], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:461..922], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:922..1383], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:1383..1842]]}, projection=[f0, f1, f2] +physical_plan DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:0..461], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:461..922], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:922..1383], [WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow:1383..1842]]}, projection=[f0, f1, f2], file_type=arrow # correct content query ITB @@ -285,7 +285,7 @@ query TT EXPLAIN SELECT * FROM avro_table ---- logical_plan TableScan: avro_table projection=[f1, f2, f3] -physical_plan AvroExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/simple_enum.avro]]}, projection=[f1, f2, f3] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/avro/simple_enum.avro]]}, projection=[f1, f2, f3], file_type=avro # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index cb363eee42ee..a6826a6ef108 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1854,7 +1854,7 @@ logical_plan 02)--TableScan: simple_string projection=[letter] physical_plan 01)ProjectionExec: expr=[letter@0 as letter, letter@0 = A as simple_string.letter = left(Utf8("APACHE"),Int64(1))] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TB SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; @@ -1873,7 +1873,7 @@ logical_plan 02)--TableScan: simple_string projection=[letter, letter2] physical_plan 01)ProjectionExec: expr=[letter@0 as letter, letter@0 = left(letter2@1, 1) as simple_string.letter = left(simple_string.letter2,Int64(1))] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TB SELECT letter, letter = LEFT(letter2, 1) FROM simple_string; diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 744aa7d9ec92..0650f978d3a8 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -424,19 +424,19 @@ query TT EXPLAIN VALUES (1, 'a', -1, 1.1),(NULL, 'b', -3, 0.5) ---- logical_plan Values: (Int64(1), Utf8("a"), Int64(-1), Float64(1.1)), (Int64(NULL), Utf8("b"), Int64(-3), Float64(0.5)) -physical_plan MemoryExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN VALUES ('1'::float) ---- logical_plan Values: (Float32(1) AS Utf8("1")) -physical_plan MemoryExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN VALUES (('1'||'2')::int unsigned) ---- logical_plan Values: (UInt32(12) AS Utf8("1") || Utf8("2")) -physical_plan MemoryExec: partitions=1, partition_sizes=[1] +physical_plan DataSourceExec: partitions=1, partition_sizes=[1] # all where empty @@ -558,15 +558,15 @@ EXPLAIN SELECT * FROM ((SELECT column1 FROM foo) "T1" CROSS JOIN (SELECT column2 ---- logical_plan 01)SubqueryAlias: F -02)--Cross Join: +02)--Cross Join: 03)----SubqueryAlias: T1 04)------TableScan: foo projection=[column1] 05)----SubqueryAlias: T2 06)------TableScan: foo projection=[column2] physical_plan 01)CrossJoinExec -02)--MemoryExec: partitions=1, partition_sizes=[1] -03)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--DataSourceExec: partitions=1, partition_sizes=[1] # select NaNs query BBBB @@ -933,7 +933,7 @@ logical_plan 02)--TableScan: select_between_data projection=[c1] physical_plan 01)ProjectionExec: expr=[c1@0 >= 2 AND c1@0 <= 3 as select_between_data.c1 BETWEEN Int64(2) AND Int64(3)] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # TODO: query_get_indexed_field @@ -1168,7 +1168,7 @@ select "INT32" from case_sensitive_table; 4 5 -# Columns in the table are a,b,c,d. Source is CsvExec which is ordered by +# Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. # Column c has cardinality 100 (unique entries). Column d has cardinality 5. statement ok @@ -1196,7 +1196,7 @@ EXPLAIN SELECT a FROM annotated_data_finite2 logical_plan 01)Sort: annotated_data_finite2.a ASC NULLS LAST, fetch=5 02)--TableScan: annotated_data_finite2 projection=[a] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], limit=5, output_ordering=[a@0 ASC NULLS LAST], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], limit=5, output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true query I SELECT a FROM annotated_data_finite2 @@ -1379,7 +1379,7 @@ logical_plan 02)--TableScan: table1 projection=[a] physical_plan 01)SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # ambiguous column references in on join query error DataFusion error: Schema error: Ambiguous reference to unqualified field a @@ -1405,7 +1405,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, a@0 + b@1 as annotated_data_finite2.a + annotated_data_finite2.b] 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true # since query below doesn't computation # inside projection expr, increasing partitions @@ -1422,7 +1422,7 @@ logical_plan 03)----TableScan: annotated_data_finite2 projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, 2 as Int64(2)] -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a @@ -1443,7 +1443,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1464,7 +1464,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1485,7 +1485,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is constant for column a and b @@ -1506,7 +1506,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # source is ordered by a,b,c # when filter result is when filter contains or @@ -1528,7 +1528,7 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: a@1 = 0 OR b@2 = 0 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true # When ordering lost during projection, we shouldn't keep the SortExec. # in the final physical plan. @@ -1554,7 +1554,7 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------ProjectionExec: expr=[c2@0 as c2] 07)------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], file_type=csv, has_header=true # FilterExec can track equality of non-column expressions. # plan below shouldn't have a SortExec because given column 'a' is ordered. @@ -1574,7 +1574,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true statement ok @@ -1602,7 +1602,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as t.x / Int64(2), __common_expr_1@0 + 1 as t.x / Int64(2) + Int64(1)] 02)--ProjectionExec: expr=[x@0 / 2 as __common_expr_1] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT x/2, x/2+1 FROM t; @@ -1620,7 +1620,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 as abs(t.x), __common_expr_1@0 + abs(y@1) as abs(t.x) + abs(t.y)] 02)--ProjectionExec: expr=[abs(x@0) as __common_expr_1, y@1 as y] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT abs(x), abs(x) + abs(y) FROM t; @@ -1658,7 +1658,7 @@ logical_plan 02)--TableScan: t projection=[x, y] physical_plan 01)ProjectionExec: expr=[coalesce(1, CAST(y@1 / x@0 AS Int64)) as coalesce(Int64(1),t.y / t.x), coalesce(2, CAST(y@1 / x@0 AS Int64)) as coalesce(Int64(2),t.y / t.x)] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT y > 0 and 1 / y < 1, x > 0 and y > 0 and 1 / y < 1 / x from t; @@ -1670,7 +1670,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 AND 1 / CAST(y@2 AS Int64) < 1 as t.y > Int64(0) AND Int64(1) / t.y < Int64(1), x@1 > 0 AND __common_expr_1@0 AND 1 / CAST(y@2 AS Int64) < 1 / CAST(x@1 AS Int64) as t.x > Int64(0) AND t.y > Int64(0) AND Int64(1) / t.y < Int64(1) / t.x] 02)--ProjectionExec: expr=[y@1 > 0 as __common_expr_1, x@0 as x, y@1 as y] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT y = 0 or 1 / y < 1, x = 0 or y = 0 or 1 / y < 1 / x from t; @@ -1682,7 +1682,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 OR 1 / CAST(y@2 AS Int64) < 1 as t.y = Int64(0) OR Int64(1) / t.y < Int64(1), x@1 = 0 OR __common_expr_1@0 OR 1 / CAST(y@2 AS Int64) < 1 / CAST(x@1 AS Int64) as t.x = Int64(0) OR t.y = Int64(0) OR Int64(1) / t.y < Int64(1) / t.x] 02)--ProjectionExec: expr=[y@1 = 0 as __common_expr_1, x@0 as x, y@1 as y] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] # due to the reason describe in https://github.com/apache/datafusion/issues/8927, # the following queries will fail diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 1df52dd1eb3d..162c9a17b61f 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -39,9 +39,9 @@ logical_plan physical_plan 01)SortMergeJoin: join_type=Inner, on=[(a@0, a@0)], filter=CAST(b@1 AS Int64) * 50 <= CAST(b@0 AS Int64) 02)--SortExec: expr=[a@0 ASC], preserve_partitioning=[false] -03)----MemoryExec: partitions=1, partition_sizes=[1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)--SortExec: expr=[a@0 ASC], preserve_partitioning=[false] -05)----MemoryExec: partitions=1, partition_sizes=[1] +05)----DataSourceExec: partitions=1, partition_sizes=[1] # inner join with join filter query TITI rowsort diff --git a/datafusion/sqllogictest/test_files/struct.slt b/datafusion/sqllogictest/test_files/struct.slt index 0afe39de1795..b547271925aa 100644 --- a/datafusion/sqllogictest/test_files/struct.slt +++ b/datafusion/sqllogictest/test_files/struct.slt @@ -124,7 +124,7 @@ logical_plan 02)--TableScan: values projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[struct(a@0, b@1, c@2) as struct(values.a,values.b,values.c)] -02)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] # error on 0 arguments query error diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 597d333d1445..b0c9ad93e155 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -209,11 +209,11 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] 11)------CoalesceBatchesExec: target_batch_size=2 12)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 13)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)------------MemoryExec: partitions=1, partition_sizes=[1] +14)------------DataSourceExec: partitions=1, partition_sizes=[1] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -245,11 +245,11 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] 11)------CoalesceBatchesExec: target_batch_size=2 12)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 13)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)------------MemoryExec: partitions=1, partition_sizes=[1] +14)------------DataSourceExec: partitions=1, partition_sizes=[1] query IR rowsort SELECT t1_id, (SELECT sum(t2_int * 1.0) + 1 FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -281,11 +281,11 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] 11)------CoalesceBatchesExec: target_batch_size=2 12)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 13)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)------------MemoryExec: partitions=1, partition_sizes=[1] +14)------------DataSourceExec: partitions=1, partition_sizes=[1] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id group by t2_id, 'a') as t2_sum from t1 @@ -320,11 +320,11 @@ physical_plan 09)----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 10)------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------MemoryExec: partitions=1, partition_sizes=[1] +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] 13)------CoalesceBatchesExec: target_batch_size=2 14)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 15)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------MemoryExec: partitions=1, partition_sizes=[1] +16)------------DataSourceExec: partitions=1, partition_sizes=[1] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 @@ -1157,11 +1157,11 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 11)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index a3717dd838d6..4ca19c0b6af7 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -43,13 +43,13 @@ EXPLAIN SELECT c1 FROM (SELECT c1 FROM sink_table ORDER BY c2) AS ttt logical_plan 01)SubqueryAlias: ttt 02)--TableScan: sink_table projection=[c1] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query TT EXPLAIN SELECT c1 FROM (SELECT c1 FROM sink_table ORDER BY c2) ---- logical_plan TableScan: sink_table projection=[c1] -physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true # Do not remove ordering when it's with limit @@ -67,7 +67,7 @@ physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] 02)--SortExec: expr=[c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST, c9@3 ASC NULLS LAST], preserve_partitioning=[false] 03)----SortExec: TopK(fetch=2), expr=[c1@0 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[false] -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true query TI @@ -102,7 +102,7 @@ physical_plan 03)----ProjectionExec: expr=[c1@0 as c1, rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as r, c3@1 as c3, c9@2 as c9] 04)------BoundedWindowAggExec: wdw=[rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Utf8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 DESC], preserve_partitioning=[false] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3, c9], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3, c9], file_type=csv, has_header=true query TT @@ -126,7 +126,7 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], file_type=csv, has_header=true query TI diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index 57a4dd95f522..b5ff95c358d8 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -85,7 +85,7 @@ logical_plan 02)--TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13] physical_plan 01)SortExec: TopK(fetch=5), expr=[c13@12 DESC], preserve_partitioning=[false] -02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index 34d63f67efdf..2616b7b75b30 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -57,4 +57,4 @@ physical_plan 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] -11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false +11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index e4376437ad4f..fee496f92055 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -89,18 +89,18 @@ physical_plan 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 20)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false +21)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], file_type=csv, has_header=false 22)----------------------------------CoalesceBatchesExec: target_batch_size=8192 23)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 24)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 25)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -26)------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +26)------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false 27)--------------------------CoalesceBatchesExec: target_batch_size=8192 28)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 29)------------------------------CoalesceBatchesExec: target_batch_size=8192 30)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -31)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false +31)----------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], file_type=csv, has_header=false 32)------------------CoalesceBatchesExec: target_batch_size=8192 33)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 34)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +35)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index 3928684a6824..b5c4e6158478 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -89,17 +89,17 @@ physical_plan 14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -17)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], has_header=false +17)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false 18)----------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 20)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +21)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 24)------------------------CoalesceBatchesExec: target_batch_size=8192 25)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] 26)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +27)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false 28)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] 29)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 30)----------CoalescePartitionsExec @@ -112,14 +112,14 @@ physical_plan 37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] 38)--------------------------CoalesceBatchesExec: target_batch_size=8192 39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -40)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false +40)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false 41)--------------------------CoalesceBatchesExec: target_batch_size=8192 42)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 43)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -44)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +44)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 45)------------------CoalesceBatchesExec: target_batch_size=8192 46)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 47)----------------------CoalesceBatchesExec: target_batch_size=8192 48)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] 49)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -50)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +50)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index 34e5dd69f410..3757fc48dba0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -72,7 +72,7 @@ physical_plan 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] -14)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false +14)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false 15)------------------CoalesceBatchesExec: target_batch_size=8192 16)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -17)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false +17)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index f584892e8aa2..eb41445c3c13 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -67,9 +67,9 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false +15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], file_type=csv, has_header=false 16)----------------------CoalesceBatchesExec: target_batch_size=8192 17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 18)--------------------------CoalesceBatchesExec: target_batch_size=8192 19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false \ No newline at end of file +20)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index 96cfbb02d07c..1104af2bdc64 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -52,8 +52,8 @@ physical_plan 09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] -12)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false +12)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false 13)--------------CoalesceBatchesExec: target_batch_size=8192 14)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +16)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 4cf4c722eaf8..e831b84d82dc 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -82,7 +82,7 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], file_type=csv, has_header=false 13)----------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] 14)------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 15)--------------------CoalesceBatchesExec: target_batch_size=8192 @@ -90,7 +90,7 @@ physical_plan 17)------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 18)--------------------------CoalesceBatchesExec: target_batch_size=8192 19)----------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +20)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false 21)--------CoalesceBatchesExec: target_batch_size=8192 22)----------RepartitionExec: partitioning=Hash([max(revenue0.total_revenue)@0], 4), input_partitions=1 23)------------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] @@ -103,4 +103,4 @@ physical_plan 30)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 31)----------------------------CoalesceBatchesExec: target_batch_size=8192 32)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -33)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +33)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index 93021d371e6f..c648f164c809 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -84,16 +84,16 @@ physical_plan 17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -20)--------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey], has_header=false +20)--------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey], file_type=csv, has_header=false 21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 22)------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 23)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) 25)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -26)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], has_header=false +26)--------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], file_type=csv, has_header=false 27)--------------------------CoalesceBatchesExec: target_batch_size=8192 28)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 29)------------------------------CoalesceBatchesExec: target_batch_size=8192 30)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] 31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false +32)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 5fc2973d6fa2..02553890bcf5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -61,16 +61,16 @@ physical_plan 08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false +11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], file_type=csv, has_header=false 12)----------------CoalesceBatchesExec: target_batch_size=8192 13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 14)--------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] 16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false +17)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], file_type=csv, has_header=false 18)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] 19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] 20)----------------CoalesceBatchesExec: target_batch_size=8192 21)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -23)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], has_header=false +23)----------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 283de4dddfe3..55da5371671e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -84,17 +84,17 @@ physical_plan 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false +18)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], file_type=csv, has_header=false 19)----------------------------CoalesceBatchesExec: target_batch_size=8192 20)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false +21)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate], file_type=csv, has_header=false 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -24)------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false +24)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false 25)----------------CoalesceBatchesExec: target_batch_size=8192 26)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] 27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] 28)----------------------CoalesceBatchesExec: target_batch_size=8192 29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -31)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false +31)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 9eb1f8638292..b0e5b2e904d0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -74,10 +74,10 @@ physical_plan 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] -11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false +11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=csv, has_header=false 12)------------CoalesceBatchesExec: target_batch_size=8192 13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 14)----------------CoalesceBatchesExec: target_batch_size=8192 15)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false +17)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index 8fdbb6884d14..2a8ee9f229b7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -126,24 +126,24 @@ physical_plan 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 26)--------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] 27)----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false +28)------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false 29)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 30)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -31)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +31)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false 32)------------------------------------CoalesceBatchesExec: target_batch_size=8192 33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 34)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false +35)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=csv, has_header=false 36)--------------------------CoalesceBatchesExec: target_batch_size=8192 37)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 38)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +39)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false 40)------------------CoalesceBatchesExec: target_batch_size=8192 41)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 42)----------------------CoalesceBatchesExec: target_batch_size=8192 43)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] 44)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +45)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false 46)----------CoalesceBatchesExec: target_batch_size=8192 47)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 48)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] @@ -163,18 +163,18 @@ physical_plan 62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] 63)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 64)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -65)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +65)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false 66)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 67)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 68)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -69)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +69)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 70)------------------------------------CoalesceBatchesExec: target_batch_size=8192 71)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 72)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -73)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +73)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false 74)----------------------------CoalesceBatchesExec: target_batch_size=8192 75)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 76)--------------------------------CoalesceBatchesExec: target_batch_size=8192 77)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] 78)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -79)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +79)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 4d0cb1bc7e1d..4844d5fae60b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -92,13 +92,13 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], file_type=csv, has_header=false 13)----------------CoalesceBatchesExec: target_batch_size=8192 14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 15)--------------------CoalesceBatchesExec: target_batch_size=8192 16)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] 17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +18)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false 19)--------CoalesceBatchesExec: target_batch_size=8192 20)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 21)------------CoalesceBatchesExec: target_batch_size=8192 @@ -109,13 +109,13 @@ physical_plan 26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] 27)------------------------CoalesceBatchesExec: target_batch_size=8192 28)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -29)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false +29)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], file_type=csv, has_header=false 30)------------------------CoalesceBatchesExec: target_batch_size=8192 31)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 32)----------------------------CoalesceBatchesExec: target_batch_size=8192 33)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] 34)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +35)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false 36)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] 37)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] 38)--------------------CoalesceBatchesExec: target_batch_size=8192 @@ -123,4 +123,4 @@ physical_plan 40)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] 41)--------------------------CoalesceBatchesExec: target_batch_size=8192 42)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -43)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false +43)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index 939e01b1784e..9e39732689da 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -116,28 +116,28 @@ physical_plan 24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false +27)----------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], file_type=csv, has_header=false 28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 31)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -32)------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +32)------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false 33)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 34)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -37)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false +37)----------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], file_type=csv, has_header=false 38)------------------------------CoalesceBatchesExec: target_batch_size=8192 39)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 40)----------------------------------CoalesceBatchesExec: target_batch_size=8192 41)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] 42)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -43)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +43)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false 44)----------------------CoalesceBatchesExec: target_batch_size=8192 45)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -46)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], has_header=false +46)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], file_type=csv, has_header=false 47)------------------CoalesceBatchesExec: target_batch_size=8192 48)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 49)----------------------CoalesceBatchesExec: target_batch_size=8192 50)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -51)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +51)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index a55d1e8fdfb1..e7b1f0a5989e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -92,14 +92,14 @@ physical_plan 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------------------FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) 18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], has_header=false +19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false 20)--------------------------CoalesceBatchesExec: target_batch_size=8192 21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -22)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], has_header=false +22)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], file_type=csv, has_header=false 23)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] 24)----------------------CoalescePartitionsExec 25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] 26)--------------------------CoalesceBatchesExec: target_batch_size=8192 27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]), projection=[c_acctbal@1] 28)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +29)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index 0f35239bfd9d..2ad496ef26fd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -76,14 +76,14 @@ physical_plan 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] 18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false +19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], file_type=csv, has_header=false 20)--------------------------CoalesceBatchesExec: target_batch_size=8192 21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 22)------------------------------CoalesceBatchesExec: target_batch_size=8192 23)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 -24)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false +24)----------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], file_type=csv, has_header=false 25)------------------CoalesceBatchesExec: target_batch_size=8192 26)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 27)----------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -29)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false +29)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index 693d37a5b53d..fb93850ab095 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -66,9 +66,9 @@ physical_plan 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] -14)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false +14)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], file_type=csv, has_header=false 15)------------------CoalesceBatchesExec: target_batch_size=8192 16)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 17)----------------------CoalesceBatchesExec: target_batch_size=8192 18)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] -19)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false +19)--------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index 0c03e0498d64..f192f987b3ef 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -95,26 +95,26 @@ physical_plan 26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +29)--------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false 30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 33)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -34)----------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +34)----------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false 35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -37)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false +37)----------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false 38)----------------------------------CoalesceBatchesExec: target_batch_size=8192 39)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 40)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -41)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +41)----------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 42)--------------------------CoalesceBatchesExec: target_batch_size=8192 43)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +45)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false 46)------------------CoalesceBatchesExec: target_batch_size=8192 47)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 48)----------------------CoalesceBatchesExec: target_batch_size=8192 49)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] 50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +51)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part index 6c818a1e2f42..b1e5d2869a8c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -40,4 +40,4 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] -07)------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false +07)------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index 023af0ea9fbd..e03de9596fbe 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -113,28 +113,28 @@ physical_plan 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 29)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +30)----------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 32)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 34)----------------------------------------------------------FilterExec: l_shipdate@4 >= 1995-01-01 AND l_shipdate@4 <= 1996-12-31 -35)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +35)------------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false 36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 37)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -38)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey], has_header=false +38)------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey], file_type=csv, has_header=false 39)------------------------------------CoalesceBatchesExec: target_batch_size=8192 40)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 41)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +42)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false 43)----------------------------CoalesceBatchesExec: target_batch_size=8192 44)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 45)--------------------------------CoalesceBatchesExec: target_batch_size=8192 46)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY 47)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +48)--------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false 49)--------------------CoalesceBatchesExec: target_batch_size=8192 50)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 51)------------------------CoalesceBatchesExec: target_batch_size=8192 52)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE 53)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -54)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +54)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 2bcab40dc985..88ceffd62ad3 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -129,34 +129,34 @@ physical_plan 37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 38)--------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] 39)----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +40)------------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_header=false 41)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 42)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -43)------------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +43)------------------------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], file_type=csv, has_header=false 44)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 46)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +47)------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 48)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 49)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 50)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 51)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -52)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +52)------------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], file_type=csv, has_header=false 53)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 54)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 55)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -56)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +56)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_header=false 57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 58)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 59)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -60)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +60)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false 61)----------------------------CoalesceBatchesExec: target_batch_size=8192 62)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 63)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -64)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +64)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false 65)--------------------CoalesceBatchesExec: target_batch_size=8192 66)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 67)------------------------CoalesceBatchesExec: target_batch_size=8192 68)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] 69)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -70)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +70)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 4a288893da95..8ccf967187d7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -106,21 +106,21 @@ physical_plan 29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 30)----------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] 31)------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)--------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +32)--------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], file_type=csv, has_header=false 33)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 34)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -35)--------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +35)--------------------------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], file_type=csv, has_header=false 36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 38)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +39)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false 40)------------------------------------CoalesceBatchesExec: target_batch_size=8192 41)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -42)----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +42)----------------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false 43)----------------------------CoalesceBatchesExec: target_batch_size=8192 44)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -45)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false +45)--------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], file_type=csv, has_header=false 46)--------------------CoalesceBatchesExec: target_batch_size=8192 47)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 48)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +49)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 5517af547542..484743fc1664 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -239,10 +239,10 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 06)----------UnionExec -07)------------MemoryExec: partitions=1, partition_sizes=[1] -08)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)------------DataSourceExec: partitions=1, partition_sizes=[1] 09)------------ProjectionExec: expr=[name@0 || _new as name] -10)--------------MemoryExec: partitions=1, partition_sizes=[1] +10)--------------DataSourceExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort @@ -270,10 +270,10 @@ logical_plan 05)----TableScan: t2 projection=[name] physical_plan 01)UnionExec -02)--MemoryExec: partitions=1, partition_sizes=[1] -03)--MemoryExec: partitions=1, partition_sizes=[1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--DataSourceExec: partitions=1, partition_sizes=[1] 04)--ProjectionExec: expr=[name@0 || _new as name] -05)----MemoryExec: partitions=1, partition_sizes=[1] +05)----DataSourceExec: partitions=1, partition_sizes=[1] # Make sure to choose a small batch size to introduce parallelism to the plan. statement ok @@ -314,12 +314,12 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)------CoalesceBatchesExec: target_batch_size=2 11)--------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 12)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] 13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------------MemoryExec: partitions=1, partition_sizes=[1] +14)--------------DataSourceExec: partitions=1, partition_sizes=[1] 15)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] 16)----CoalesceBatchesExec: target_batch_size=2 17)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] @@ -331,11 +331,11 @@ physical_plan 23)------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 24)--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 25)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -26)------------------------MemoryExec: partitions=1, partition_sizes=[1] +26)------------------------DataSourceExec: partitions=1, partition_sizes=[1] 27)--------CoalesceBatchesExec: target_batch_size=2 28)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 29)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)--------------MemoryExec: partitions=1, partition_sizes=[1] +30)--------------DataSourceExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -388,11 +388,11 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)------CoalesceBatchesExec: target_batch_size=2 11)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 12)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)------------MemoryExec: partitions=1, partition_sizes=[1] +13)------------DataSourceExec: partitions=1, partition_sizes=[1] 14)--CoalesceBatchesExec: target_batch_size=2 15)----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] 16)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] @@ -400,11 +400,11 @@ physical_plan 18)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 19)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 20)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)----------------MemoryExec: partitions=1, partition_sizes=[1] +21)----------------DataSourceExec: partitions=1, partition_sizes=[1] 22)------CoalesceBatchesExec: target_batch_size=2 23)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 24)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -25)------------MemoryExec: partitions=1, partition_sizes=[1] +25)------------DataSourceExec: partitions=1, partition_sizes=[1] # union_upcast_types query TT @@ -423,11 +423,11 @@ physical_plan 03)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[c1@0 as c1, CAST(c9@1 AS Decimal128(20, 0)) as c9] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true 07)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] 08)------ProjectionExec: expr=[c1@0 as c1, CAST(c3@1 AS Decimal128(20, 0)) as c9] 09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true +10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true query TR SELECT c1, c9 FROM aggregate_test_100 UNION ALL SELECT c1, c3 FROM aggregate_test_100 ORDER BY c9 DESC LIMIT 5 @@ -464,13 +464,13 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[1] 10)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 11)--------CoalesceBatchesExec: target_batch_size=2 12)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 13)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 14)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------MemoryExec: partitions=1, partition_sizes=[1] +15)----------------DataSourceExec: partitions=1, partition_sizes=[1] # Union with limit push down 3 children test case query TT @@ -524,7 +524,7 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=2 13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true 16)----ProjectionExec: expr=[1 as cnt] 17)------PlaceholderRowExec 18)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] @@ -609,9 +609,9 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] 02)--UnionExec -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true 04)----ProjectionExec: expr=[c1a@0 as c1] -05)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1a], output_ordering=[c1a@0 ASC NULLS LAST], has_header=true +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1a], output_ordering=[c1a@0 ASC NULLS LAST], file_type=csv, has_header=true statement ok drop table t1 @@ -826,11 +826,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------FilterExec: c1@0 = a 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true 07)----CoalesceBatchesExec: target_batch_size=2 08)------FilterExec: c1@0 = a 09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], has_header=true +10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true # Clean up after the test statement ok diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index bc213d7739e7..9c46410c4909 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -605,7 +605,7 @@ physical_plan 03)----ProjectionExec: expr=[__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0 as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), column3@1 as column3] 04)------UnnestExec 05)--------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] ## unnest->field_access->unnest->unnest query I? @@ -663,7 +663,7 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------UnnestExec 06)----------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] -07)------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 679e0f637f4b..ca4713e7d516 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1245,7 +1245,7 @@ physical_plan 03)----ProjectionExec: expr=[c9@1 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c9@1 ASC NULLS LAST, c8@0 ASC NULLS LAST], preserve_partitioning=[false] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], file_type=csv, has_header=true # over_order_by_sort_keys_sorting_prefix_compacting @@ -1266,7 +1266,7 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, has_header=true # FIXME: for now we are not detecting prefix of sorting keys in order to re-arrange with global and save one SortExec @@ -1291,7 +1291,7 @@ physical_plan 05)--------SortExec: expr=[c9@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 06)----------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 07)------------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], file_type=csv, has_header=true # test_window_partition_by_order_by statement ok @@ -1321,7 +1321,7 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=4096 10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], file_type=csv, has_header=true # test_window_agg_sort_reversed_plan @@ -1346,7 +1346,7 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query III SELECT @@ -1387,7 +1387,7 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, lag(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "lag(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }, lead(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "lead(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "first_value(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }, lag(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, lead(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query IIIIIII SELECT @@ -1431,7 +1431,7 @@ physical_plan 04)------SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false] 05)--------BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 06)----------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query III @@ -1474,7 +1474,7 @@ physical_plan 05)--------BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 07)------------SortExec: expr=[c9@2 DESC, c1@0 DESC], preserve_partitioning=[false] -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], file_type=csv, has_header=true query IIII SELECT @@ -1565,7 +1565,7 @@ physical_plan 14)--------------------------WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] 15)----------------------------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 16)------------------------------SortExec: expr=[c3@2 DESC, c1@0 ASC NULLS LAST], preserve_partitioning=[false] -17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true +17)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -1640,7 +1640,7 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[false] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true query III @@ -1684,7 +1684,7 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[false] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true query III SELECT @@ -1734,7 +1734,7 @@ physical_plan 07)------------SortExec: expr=[__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------ProjectionExec: expr=[c3@1 + c4@2 as __common_expr_1, c2@0 as c2, c3@1 as c3, c9@3 as c9] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], file_type=csv, has_header=true query III @@ -1785,7 +1785,7 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=4096 11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +13)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true query I @@ -1830,7 +1830,7 @@ physical_plan 08)--------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 09)----------------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 10)------------------SortExec: expr=[c3@1 DESC, c9@2 DESC, c2@0 ASC NULLS LAST], preserve_partitioning=[false] -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], file_type=csv, has_header=true @@ -1869,7 +1869,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=4096 06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true query TI SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC @@ -1998,7 +1998,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=4096 06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true statement ok set datafusion.optimizer.repartition_sorts = true; @@ -2027,7 +2027,7 @@ physical_plan 08)--------------CoalesceBatchesExec: target_batch_size=4096 09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], file_type=csv, has_header=true # test_window_agg_with_global_limit statement ok @@ -2048,7 +2048,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(aggregate_test_100.c13)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST], preserve_partitioning=[false] -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], file_type=csv, has_header=true query ? @@ -2113,7 +2113,7 @@ physical_plan 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 07)------------WindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] 08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, c9@3 ASC NULLS LAST, c8@2 ASC NULLS LAST], preserve_partitioning=[false] -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true @@ -2170,7 +2170,7 @@ physical_plan 08)--------------WindowAggExec: wdw=[sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] 09)----------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, c9@3 ASC NULLS LAST, c8@2 ASC NULLS LAST], preserve_partitioning=[false] 10)------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], file_type=csv, has_header=true query IIIII SELECT c9, @@ -2212,7 +2212,7 @@ physical_plan 05)--------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], file_type=csv, has_header=true query RR SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum1, @@ -2246,7 +2246,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2283,7 +2283,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2320,7 +2320,7 @@ physical_plan 02)--ProjectionExec: expr=[c9@0 as c9, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2360,7 +2360,7 @@ physical_plan 02)--ProjectionExec: expr=[c9@0 as c9, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2435,7 +2435,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true # This test shows that ordering equivalence can keep track of complex expressions (not just Column expressions) # during ordering satisfy analysis. In the final plan we should only see single SortExec. @@ -2457,7 +2457,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[CAST(c9@1 AS Int32) + c5@0 DESC], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], file_type=csv, has_header=true # Ordering equivalence should be preserved during cast expression query TT @@ -2478,7 +2478,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true # The following query has type error. We should test the error could be detected # from either the logical plan (when `skip_failed_rules` is set to `false`) or @@ -2508,7 +2508,7 @@ SELECT # 100 rows. Columns in the table are ts, inc_col, desc_col. -# Source is CsvExec which is ordered by ts column. +# Source is DataSourceExec which is ordered by ts column. # Normal, non infinite source statement ok CREATE EXTERNAL TABLE annotated_data_finite ( @@ -2522,7 +2522,7 @@ LOCATION '../core/tests/data/window_1.csv' OPTIONS ('format.has_header' 'true'); # 100 rows. Columns in the table are ts, inc_col, desc_col. -# Source is CsvExec which is ordered by ts column. +# Source is DataSourceExec which is ordered by ts column. # Infinite source statement ok CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite ( @@ -2586,7 +2586,7 @@ physical_plan 06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 07)------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)), is_causal: false }, count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] 08)--------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as __common_expr_1, CAST(inc_col@1 AS Int64) as __common_expr_2, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col, desc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col, desc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2669,7 +2669,7 @@ physical_plan 02)--ProjectionExec: expr=[ts@0 as ts, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] 03)----BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "dense_rank() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lag(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "lead(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }], mode=[Sorted] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2742,7 +2742,7 @@ physical_plan 04)------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }], mode=[Sorted] 05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "max(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }, avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "avg(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)), is_causal: false }], mode=[Sorted] 06)----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as __common_expr_1, CAST(inc_col@1 AS Float64) as __common_expr_2, ts@0 as ts, inc_col@1 as inc_col] -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIIIIIIRR SELECT @@ -2793,7 +2793,7 @@ physical_plan 03)----ProjectionExec: expr=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] 04)------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "nth_value(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 05)--------BoundedWindowAggExec: wdw=[first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "first_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }, last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "last_value(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], file_type=csv, has_header=true query IIIII SELECT @@ -2916,7 +2916,7 @@ drop table annotated_data_infinite -# Columns in the table are a,b,c,d. Source is CsvExec which is ordered by +# Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. # Column c has cardinality 100 (unique entries). Column d has cardinality 5. statement ok @@ -2932,7 +2932,7 @@ WITH ORDER (a ASC, b ASC, c ASC) LOCATION '../core/tests/data/window_2.csv' OPTIONS ('format.has_header' 'true'); -# Columns in the table are a,b,c,d. Source is CsvExec which is ordered by +# Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. # Column c has cardinality 100 (unique entries). Column d has cardinality 5. statement ok @@ -3062,7 +3062,7 @@ physical_plan 12)----------------------SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 13)------------------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 14)--------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true query IIIIIIIIIIIIIII SELECT a, b, c, @@ -3148,7 +3148,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true # Query below should work when its input is unbounded # because ordering of ROW_NUMBER, RANK result is added to the ordering equivalence @@ -3336,7 +3336,7 @@ physical_plan 05)--------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] 06)----------WindowAggExec: wdw=[min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "min(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 07)------------SortExec: expr=[c11@1 ASC NULLS LAST], preserve_partitioning=[false] -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], file_type=csv, has_header=true # window1 spec is used multiple times under different aggregations. # The query should still work. @@ -3377,7 +3377,7 @@ physical_plan 03)----ProjectionExec: expr=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] 04)------BoundedWindowAggExec: wdw=[max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }, min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c12@1 ASC NULLS LAST], preserve_partitioning=[false] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], file_type=csv, has_header=true # window2 spec is not defined statement error DataFusion error: Error during planning: The window window2 is not defined! @@ -3430,7 +3430,7 @@ physical_plan 01)BoundedWindowAggExec: wdw=[sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 02)--CoalesceBatchesExec: target_batch_size=4096 03)----FilterExec: b@2 = 0 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], file_type=csv, has_header=true # Since column b is constant after filter b=0, # window requirement b ASC, d ASC can be satisfied @@ -3449,7 +3449,7 @@ physical_plan 02)--SortExec: expr=[d@4 ASC NULLS LAST], preserve_partitioning=[false] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------FilterExec: b@2 = 0 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], file_type=csv, has_header=true # Create an unbounded source where there is multiple orderings. @@ -3485,7 +3485,7 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[min(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----ProjectionExec: expr=[c@2 as c, d@3 as d, max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 04)------BoundedWindowAggExec: wdw=[max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], file_type=csv, has_header=true query TT EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c @@ -3504,7 +3504,7 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------FilterExec: d@1 = 0 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) @@ -3517,7 +3517,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[sum(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 02)--BoundedWindowAggExec: wdw=[sum(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], file_type=csv, has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3530,7 +3530,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[sum(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 02)--BoundedWindowAggExec: wdw=[sum(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], file_type=csv, has_header=true query I SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3572,7 +3572,7 @@ physical_plan 01)ProjectionExec: expr=[c@0 as c, nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] 02)--GlobalLimitExec: skip=0, fetch=5 03)----WindowAggExec: wdw=[nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "nth_value(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)), is_causal: false }] -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], file_type=csv, has_header=true query II SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 @@ -3959,7 +3959,7 @@ physical_plan 01)ProjectionExec: expr=[sn@0 as sn, ts@1 as ts, currency@2 as currency, amount@3 as amount, sum(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum1] 02)--BoundedWindowAggExec: wdw=[sum(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] 03)----SortExec: expr=[sn@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------MemoryExec: partitions=1, partition_sizes=[1] +04)------DataSourceExec: partitions=1, partition_sizes=[1] # test ROW_NUMBER window function returns correct data_type query T @@ -4080,7 +4080,7 @@ physical_plan 04)------ProjectionExec: expr=[c3@0 as c3, c4@1 as c4, c9@2 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 06)----------SortExec: expr=[c3@0 + c4@1 DESC], preserve_partitioning=[false] -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], file_type=csv, has_header=true query III SELECT c3, @@ -4123,7 +4123,7 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=4096 07)------------FilterExec: a@0 = 1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4146,7 +4146,7 @@ physical_plan 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=4096 07)------------FilterExec: a@0 = 1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------DataSourceExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT @@ -5209,7 +5209,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------FilterExec: c1@0 = 2 OR c1@0 = 3 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5251,7 +5251,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5295,7 +5295,7 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=1 10)------------------FilterExec: c1@0 = 1 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------MemoryExec: partitions=1, partition_sizes=[1] +12)----------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5334,7 +5334,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=1 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] query III select c1, c2, rank @@ -5383,7 +5383,7 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=1 13)------------------------FilterExec: c1@0 > 1 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------MemoryExec: partitions=1, partition_sizes=[1] +15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5434,7 +5434,7 @@ physical_plan 12)----------------------CoalesceBatchesExec: target_batch_size=1 13)------------------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------MemoryExec: partitions=1, partition_sizes=[1] +15)----------------------------DataSourceExec: partitions=1, partition_sizes=[1] query IIII select c1, c2, rank1, rank2 @@ -5494,7 +5494,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=1 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT SUM(c9) OVER() as sum_c9 FROM aggregate_test_100_ordered ORDER BY sum_c9; @@ -5507,7 +5507,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as sum_c9] 02)--WindowAggExec: wdw=[sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100_ordered.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true query TT EXPLAIN SELECT c1, MIN(c5) OVER(PARTITION BY c1) as min_c5 FROM aggregate_test_100_ordered ORDER BY c1, min_c5 DESC NULLS LAST; @@ -5524,7 +5524,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=1 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2, preserve_order=true, sort_exprs=c1@0 ASC NULLS LAST 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], has_header=true +07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c5], output_ordering=[c1@0 ASC NULLS LAST], file_type=csv, has_header=true query TT EXPLAIN SELECT MAX(c5) OVER() as max_c5 FROM aggregate_test_100_ordered ORDER BY max_c5; @@ -5537,4 +5537,4 @@ logical_plan physical_plan 01)ProjectionExec: expr=[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as max_c5] 02)--WindowAggExec: wdw=[max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "max(aggregate_test_100_ordered.c5) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5], file_type=csv, has_header=true diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index f4548df4309e..ce056ddac664 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -22,7 +22,7 @@ use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::common::{not_impl_err, substrait_err}; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; @@ -51,6 +51,7 @@ pub async fn from_substrait_rel( ) -> Result> { let mut base_config; + let source = Arc::new(ParquetSource::default()); match &rel.rel_type { Some(RelType::Read(read)) => { if read.filter.is_some() || read.best_effort_filter.is_some() { @@ -80,6 +81,7 @@ pub async fn from_substrait_rel( base_config = FileScanConfig::new( ObjectStoreUrl::local_filesystem(), Arc::new(Schema::new(fields)), + source, ); } Err(e) => return Err(e), @@ -150,8 +152,7 @@ pub async fn from_substrait_rel( } } - Ok(ParquetExec::builder(base_config).build_arc() - as Arc) + Ok(base_config.new_exec() as Arc) } _ => not_impl_err!( "Only LocalFile reads are supported when parsing physical" diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 7279785ae873..3fc94a33442b 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -15,11 +15,19 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; + +use crate::variation_const::{ + DEFAULT_CONTAINER_TYPE_VARIATION_REF, LARGE_CONTAINER_TYPE_VARIATION_REF, + VIEW_CONTAINER_TYPE_VARIATION_REF, +}; + use datafusion::arrow::datatypes::DataType; -use datafusion::datasource::physical_plan::ParquetExec; use datafusion::error::{DataFusionError, Result}; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::{displayable, ExecutionPlan}; -use std::collections::HashMap; + +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; use substrait::proto::expression::MaskExpression; use substrait::proto::r#type::{ @@ -35,11 +43,6 @@ use substrait::proto::ReadRel; use substrait::proto::Rel; use substrait::proto::{extensions, NamedStruct, Type}; -use crate::variation_const::{ - DEFAULT_CONTAINER_TYPE_VARIATION_REF, LARGE_CONTAINER_TYPE_VARIATION_REF, - VIEW_CONTAINER_TYPE_VARIATION_REF, -}; - /// Convert DataFusion ExecutionPlan to Substrait Rel pub fn to_substrait_rel( plan: &dyn ExecutionPlan, @@ -48,87 +51,96 @@ pub fn to_substrait_rel( HashMap, ), ) -> Result> { - if let Some(scan) = plan.as_any().downcast_ref::() { - let base_config = scan.base_config(); - let mut substrait_files = vec![]; - for (partition_index, files) in base_config.file_groups.iter().enumerate() { - for file in files { - substrait_files.push(FileOrFiles { - partition_index: partition_index.try_into().unwrap(), - start: 0, - length: file.object_meta.size as u64, - path_type: Some(PathType::UriPath( - file.object_meta.location.as_ref().to_string(), - )), - file_format: Some(FileFormat::Parquet(ParquetReadOptions {})), - }); - } - } + if let Some(data_source) = plan.as_any().downcast_ref::() { + let source = data_source.source(); + if let Some(file_config) = source.as_any().downcast_ref::() { + let is_parquet = file_config + .file_source() + .as_any() + .downcast_ref::() + .is_some(); + if is_parquet { + let mut substrait_files = vec![]; + for (partition_index, files) in file_config.file_groups.iter().enumerate() + { + for file in files { + substrait_files.push(FileOrFiles { + partition_index: partition_index.try_into().unwrap(), + start: 0, + length: file.object_meta.size as u64, + path_type: Some(PathType::UriPath( + file.object_meta.location.as_ref().to_string(), + )), + file_format: Some(FileFormat::Parquet(ParquetReadOptions {})), + }); + } + } - let mut names = vec![]; - let mut types = vec![]; + let mut names = vec![]; + let mut types = vec![]; - for field in base_config.file_schema.fields.iter() { - match to_substrait_type(field.data_type(), field.is_nullable()) { - Ok(t) => { - names.push(field.name().clone()); - types.push(t); + for field in file_config.file_schema.fields.iter() { + match to_substrait_type(field.data_type(), field.is_nullable()) { + Ok(t) => { + names.push(field.name().clone()); + types.push(t); + } + Err(e) => return Err(e), + } } - Err(e) => return Err(e), - } - } - let type_info = Struct { - types, - // FIXME: duckdb doesn't set this field, keep it as default variant 0. - // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1106-L1127 - type_variation_reference: 0, - nullability: Nullability::Required.into(), - }; + let type_info = Struct { + types, + // FIXME: duckdb doesn't set this field, keep it as default variant 0. + // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1106-L1127 + type_variation_reference: 0, + nullability: Nullability::Required.into(), + }; - let mut select_struct = None; - if let Some(projection) = base_config.projection.as_ref() { - let struct_items = projection - .iter() - .map(|index| StructItem { - field: *index as i32, - // FIXME: duckdb sets this to None, but it's not clear why. - // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1191 - child: None, - }) - .collect(); + let mut select_struct = None; + if let Some(projection) = file_config.projection.as_ref() { + let struct_items = projection + .iter() + .map(|index| StructItem { + field: *index as i32, + // FIXME: duckdb sets this to None, but it's not clear why. + // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1191 + child: None, + }) + .collect(); - select_struct = Some(StructSelect { struct_items }); - } + select_struct = Some(StructSelect { struct_items }); + } - Ok(Box::new(Rel { - rel_type: Some(RelType::Read(Box::new(ReadRel { - common: None, - base_schema: Some(NamedStruct { - names, - r#struct: Some(type_info), - }), - filter: None, - best_effort_filter: None, - projection: Some(MaskExpression { - select: select_struct, - // FIXME: duckdb set this to true, but it's not clear why. - // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1186. - maintain_singular_struct: true, - }), - advanced_extension: None, - read_type: Some(ReadType::LocalFiles(LocalFiles { - items: substrait_files, - advanced_extension: None, - })), - }))), - })) - } else { - Err(DataFusionError::Substrait(format!( - "Unsupported plan in Substrait physical plan producer: {}", - displayable(plan).one_line() - ))) + return Ok(Box::new(Rel { + rel_type: Some(RelType::Read(Box::new(ReadRel { + common: None, + base_schema: Some(NamedStruct { + names, + r#struct: Some(type_info), + }), + filter: None, + best_effort_filter: None, + projection: Some(MaskExpression { + select: select_struct, + // FIXME: duckdb set this to true, but it's not clear why. + // https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L1186. + maintain_singular_struct: true, + }), + advanced_extension: None, + read_type: Some(ReadType::LocalFiles(LocalFiles { + items: substrait_files, + advanced_extension: None, + })), + }))), + })); + } + } } + Err(DataFusionError::Substrait(format!( + "Unsupported plan in Substrait physical plan producer: {}", + displayable(plan).one_line() + ))) } // see https://github.com/duckdb/substrait/blob/b6f56643cb11d52de0e32c24a01dfd5947df62be/src/to_substrait.cpp#L954-L1094. diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 57fb3e2ee7cc..04c5e8ada758 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -22,7 +22,7 @@ use datafusion::arrow::datatypes::Schema; use datafusion::dataframe::DataFrame; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::error::Result; use datafusion::physical_plan::{displayable, ExecutionPlan}; use datafusion::prelude::{ParquetReadOptions, SessionContext}; @@ -32,9 +32,12 @@ use substrait::proto::extensions; #[tokio::test] async fn parquet_exec() -> Result<()> { + let source = Arc::new(ParquetSource::default()); + let scan_config = FileScanConfig::new( ObjectStoreUrl::local_filesystem(), Arc::new(Schema::empty()), + source, ) .with_file_groups(vec![ vec![PartitionedFile::new( @@ -46,8 +49,7 @@ async fn parquet_exec() -> Result<()> { 123, )], ]); - let parquet_exec: Arc = - ParquetExec::builder(scan_config).build_arc(); + let parquet_exec: Arc = scan_config.new_exec(); let mut extension_info: ( Vec, diff --git a/docs/source/library-user-guide/building-logical-plans.md b/docs/source/library-user-guide/building-logical-plans.md index 556deb02e980..e1e75b3e4bdb 100644 --- a/docs/source/library-user-guide/building-logical-plans.md +++ b/docs/source/library-user-guide/building-logical-plans.md @@ -201,7 +201,7 @@ async fn main() -> Result<(), DataFusionError> { This example produces the following physical plan: ```text -MemoryExec: partitions=0, partition_sizes=[] +DataSourceExec: partitions=0, partition_sizes=[] ``` ## Table Sources diff --git a/docs/source/library-user-guide/custom-table-providers.md b/docs/source/library-user-guide/custom-table-providers.md index f86cea0bda95..a7183fb3113e 100644 --- a/docs/source/library-user-guide/custom-table-providers.md +++ b/docs/source/library-user-guide/custom-table-providers.md @@ -89,7 +89,7 @@ This: 2. Constructs the individual output arrays (columns) 3. Returns a `MemoryStream` of a single `RecordBatch` with the arrays -I.e. returns the "physical" data. For other examples, refer to the [`CsvExec`][csv] and [`ParquetExec`][parquet] for more complex implementations. +I.e. returns the "physical" data. For other examples, refer to the [`CsvSource`][csv] and [`ParquetSource`][parquet] for more complex implementations. With the `ExecutionPlan` implemented, we can now implement the `scan` method of the `TableProvider`. diff --git a/docs/source/user-guide/explain-usage.md b/docs/source/user-guide/explain-usage.md index 2eb03aad2ef9..32a87ae9198d 100644 --- a/docs/source/user-guide/explain-usage.md +++ b/docs/source/user-guide/explain-usage.md @@ -50,20 +50,20 @@ LIMIT 5; The output will look like ``` -+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| plan_type | plan | -+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | Sort: wid ASC NULLS LAST, ip DESC NULLS FIRST, fetch=5 | -| | Projection: hits.parquet.WatchID AS wid, hits.parquet.ClientIP AS ip | -| | Filter: starts_with(hits.parquet.URL, Utf8("http://domcheloveplanet.ru/")) | -| | TableScan: hits.parquet projection=[WatchID, ClientIP, URL], partial_filters=[starts_with(hits.parquet.URL, Utf8("http://domcheloveplanet.ru/"))] | -| physical_plan | SortPreservingMergeExec: [wid@0 ASC NULLS LAST,ip@1 DESC], fetch=5 | -| | SortExec: TopK(fetch=5), expr=[wid@0 ASC NULLS LAST,ip@1 DESC], preserve_partitioning=[true] | -| | ProjectionExec: expr=[WatchID@0 as wid, ClientIP@1 as ip] | -| | CoalesceBatchesExec: target_batch_size=8192 | -| | FilterExec: starts_with(URL@2, http://domcheloveplanet.ru/) | -| | ParquetExec: file_groups={16 groups: [[hits.parquet:0..923748528], ...]}, projection=[WatchID, ClientIP, URL], predicate=starts_with(URL@13, http://domcheloveplanet.ru/) | -+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: wid ASC NULLS LAST, ip DESC NULLS FIRST, fetch=5 | +| | Projection: hits.parquet.WatchID AS wid, hits.parquet.ClientIP AS ip | +| | Filter: starts_with(hits.parquet.URL, Utf8("http://domcheloveplanet.ru/")) | +| | TableScan: hits.parquet projection=[WatchID, ClientIP, URL], partial_filters=[starts_with(hits.parquet.URL, Utf8("http://domcheloveplanet.ru/"))] | +| physical_plan | SortPreservingMergeExec: [wid@0 ASC NULLS LAST,ip@1 DESC], fetch=5 | +| | SortExec: TopK(fetch=5), expr=[wid@0 ASC NULLS LAST,ip@1 DESC], preserve_partitioning=[true] | +| | ProjectionExec: expr=[WatchID@0 as wid, ClientIP@1 as ip] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | FilterExec: starts_with(URL@2, http://domcheloveplanet.ru/) | +| | DataSourceExec: file_groups={16 groups: [[hits.parquet:0..923748528], ...]}, projection=[WatchID, ClientIP, URL], predicate=starts_with(URL@13, http://domcheloveplanet.ru/), file_type=parquet | ++---------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 2 row(s) fetched. Elapsed 0.060 seconds. ``` @@ -123,7 +123,7 @@ physical plan in Figure 1 in tree format will look like ▲ │ ┌────────────────────────────────────────────────┐ -│ ParquetExec │ +│ DataSourceExec │ │ hits.parquet (filter = ...) │ └────────────────────────────────────────────────┘ ``` @@ -131,7 +131,7 @@ physical plan in Figure 1 in tree format will look like Each node in the tree/plan ends with `Exec` and is sometimes also called an `operator` or `ExecutionPlan` where data is processed, transformed and sent up. -1. First, data in parquet the `hits.parquet` file us read in parallel using 16 cores in 16 "partitions" (more on this later) from `ParquetExec`, which applies a first pass at filtering during the scan. +1. First, data in parquet the `hits.parquet` file us read in parallel using 16 cores in 16 "partitions" (more on this later) from `DataSourceExec`, which applies a first pass at filtering during the scan. 2. Next, the output is filtered using `FilterExec` to ensure only rows where `starts_with(URL, 'http://domcheloveplanet.ru/')` evaluates to true are passed on 3. The `CoalesceBatchesExec` then ensures that the data is grouped into larger batches for processing 4. The `ProjectionExec` then projects the data to rename the `WatchID` and `ClientIP` columns to `wid` and `ip` respectively. @@ -187,7 +187,7 @@ LIMIT 5; | | SortExec: TopK(fetch=5), expr=[wid@0 ASC NULLS LAST,ip@1 DESC], preserve_partitioning=[true], metrics=[output_rows=75, elapsed_compute=7.243038ms, row_replacements=482] | | | ProjectionExec: expr=[WatchID@0 as wid, ClientIP@1 as ip], metrics=[output_rows=811821, elapsed_compute=66.25µs] | | | FilterExec: starts_with(URL@2, http://domcheloveplanet.ru/), metrics=[output_rows=811821, elapsed_compute=1.36923816s] | -| | ParquetExec: file_groups={16 groups: [[hits.parquet:0..923748528], ...]}, projection=[WatchID, ClientIP, URL], predicate=starts_with(URL@13, http://domcheloveplanet.ru/), metrics=[output_rows=99997497, elapsed_compute=16ns, ... bytes_scanned=3703192723, ... time_elapsed_opening=308.203002ms, time_elapsed_scanning_total=8.350342183s, ...] | +| | DataSourceExec: file_groups={16 groups: [[hits.parquet:0..923748528], ...]}, projection=[WatchID, ClientIP, URL], predicate=starts_with(URL@13, http://domcheloveplanet.ru/), metrics=[output_rows=99997497, elapsed_compute=16ns, ... bytes_scanned=3703192723, ... time_elapsed_opening=308.203002ms, time_elapsed_scanning_total=8.350342183s, ...] | +-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 1 row(s) fetched. Elapsed 0.720 seconds. @@ -197,14 +197,14 @@ In this case, DataFusion actually ran the query, but discarded any results, and instead returned an annotated plan with a new field, `metrics=[...]` Most operators have the common metrics `output_rows` and `elapsed_compute` and -some have operator specific metrics such as `ParquetExec` which has +some have operator specific metrics such as `DataSourceExec` with `ParquetSource` which has `bytes_scanned=3703192723`. Note that times and counters are reported across all cores, so if you have 16 cores, the time reported is the sum of the time taken by all 16 cores. Again, reading from bottom up: -- `ParquetExec` +- `DataSourceExec` - `output_rows=99997497`: A total 99.9M rows were produced - `bytes_scanned=3703192723`: Of the 14GB file, 3.7GB were actually read (due to projection pushdown) - `time_elapsed_opening=308.203002ms`: It took 300ms to open the file and prepare to read it @@ -223,7 +223,7 @@ Again, reading from bottom up: - `SortPreservingMergeExec` - `output_rows=5`, `elapsed_compute=2.375µs`: Produced the final 5 rows in 2.375µs (microseconds) -When predicate pushdown is enabled, `ParquetExec` gains the following metrics: +When predicate pushdown is enabled, `DataSourceExec` with `ParquetSource` gains the following metrics: - `page_index_rows_matched`: number of rows in pages that were tested by a page index filter, and passed - `page_index_rows_pruned`: number of rows in pages that were tested by a page index filter, and did not pass @@ -247,7 +247,7 @@ planning. Roughly speaking, each "partition" in the plan is run independently us a separate core. Data crosses between cores only within certain operators such as `RepartitionExec`, `CoalescePartitions` and `SortPreservingMergeExec` -You can read more about this in the [Partitoning Docs]. +You can read more about this in the [Partitioning Docs]. [partitoning docs]: https://docs.rs/datafusion/latest/datafusion/physical_expr/enum.Partitioning.html @@ -283,7 +283,7 @@ We can again see the query plan by using `EXPLAIN`: | | CoalesceBatchesExec: target_batch_size=8192 | | | RepartitionExec: partitioning=Hash([UserID@0], 10), input_partitions=10 | | | AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(*)] | -| | ParquetExec: file_groups={10 groups: [[hits.parquet:0..1477997645], [hits.parquet:1477997645..2955995290], [hits.parquet:2955995290..4433992935], [hits.parquet:4433992935..5911990580], [hits.parquet:5911990580..7389988225], ...]}, projection=[UserID] | +| | DataSourceExec: file_groups={10 groups: [[hits.parquet:0..1477997645], [hits.parquet:1477997645..2955995290], [hits.parquet:2955995290..4433992935], [hits.parquet:4433992935..5911990580], [hits.parquet:5911990580..7389988225], ...]}, projection=[UserID], file_type=parquet | | | | +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ``` @@ -307,11 +307,11 @@ For this query, let's again read the plan from the bottom to the top: **Physical plan operators** -- `ParquetExec` +- `DataSourceExec` - `file_groups={10 groups: [...]}`: Reads 10 groups in parallel from `hits.parquet`file. (The example above was run on a machine with 10 cores.) - `projection=[UserID]`: Pushes down projection of the `UserID` column. The parquet format is columnar and the DataFusion reader only decodes the columns required. - `AggregateExec` - - `mode=Partial` Runs a [partial aggregation] in parallel across each of the 10 partitions from the `ParquetExec` immediately after reading. + - `mode=Partial` Runs a [partial aggregation] in parallel across each of the 10 partitions from the `DataSourceExec` immediately after reading. - `gby=[UserID@0 as UserID]`: Represents `GROUP BY` in the [physical plan] and groups together the same values of `UserID`. - `aggr=[count(*)]`: Applies the `COUNT` aggregate on all rows for each group. - `RepartitionExec` diff --git a/docs/source/user-guide/sql/explain.md b/docs/source/user-guide/sql/explain.md index 45bb3a57aa7c..709e6311c28e 100644 --- a/docs/source/user-guide/sql/explain.md +++ b/docs/source/user-guide/sql/explain.md @@ -46,7 +46,7 @@ EXPLAIN SELECT SUM(x) FROM table GROUP BY b; | | RepartitionExec: partitioning=Hash([Column { name: "b", index: 0 }], 16) | | | AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[SUM(table.x)] | | | RepartitionExec: partitioning=RoundRobinBatch(16) | -| | CsvExec: file_groups={1 group: [[/tmp/table.csv]]}, projection=[x, b], has_header=false | +| | DataSourceExec: file_groups={1 group: [[/tmp/table.csv]]}, projection=[x, b], has_header=false | | | | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------+ ``` @@ -68,6 +68,6 @@ EXPLAIN ANALYZE SELECT SUM(x) FROM table GROUP BY b; | | RepartitionExec: partitioning=Hash([Column { name: "b", index: 0 }], 16), metrics=[sendTime=839560, fetchTime=122528525, repartitionTime=5327877] | | | HashAggregateExec: mode=Partial, gby=[b@1 as b], aggr=[SUM(x)], metrics=[outputRows=2] | | | RepartitionExec: partitioning=RoundRobinBatch(16), metrics=[fetchTime=5660489, repartitionTime=0, sendTime=8012] | -| | CsvExec: file_groups={1 group: [[/tmp/table.csv]]}, has_header=false, metrics=[] | +| | DataSourceExec: file_groups={1 group: [[/tmp/table.csv]]}, has_header=false, metrics=[] | +-------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------+ ```