From 5b1714c89fe82d7580c08932d18200b7a30720da Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 7 Jan 2025 16:44:06 +0300 Subject: [PATCH 01/39] unify ParquetExec, AvroExec, ArrowExec, NDJsonExec, MemoryExec into one DataSourceExec plan --- datafusion-cli/src/functions.rs | 6 +- .../examples/advanced_parquet_index.rs | 64 +- .../examples/csv_json_opener.rs | 26 +- .../examples/parquet_exec_visitor.rs | 31 +- datafusion-examples/examples/parquet_index.rs | 20 +- .../examples/remote_catalog.rs | 8 +- datafusion-examples/examples/simple_udtf.rs | 8 +- datafusion/core/benches/physical_plan.rs | 18 +- datafusion/core/benches/sort.rs | 22 +- datafusion/core/src/dataframe/mod.rs | 8 +- datafusion/core/src/datasource/data_source.rs | 364 +++++++++ .../core/src/datasource/file_format/arrow.rs | 10 +- .../core/src/datasource/file_format/avro.rs | 26 +- .../core/src/datasource/file_format/csv.rs | 35 +- .../core/src/datasource/file_format/json.rs | 14 +- .../src/datasource/file_format/parquet.rs | 26 +- datafusion/core/src/datasource/memory.rs | 20 +- datafusion/core/src/datasource/mod.rs | 1 + .../datasource/physical_plan/arrow_file.rs | 191 +---- .../core/src/datasource/physical_plan/avro.rs | 279 +++---- .../core/src/datasource/physical_plan/csv.rs | 677 ++++------------ .../physical_plan/file_scan_config.rs | 33 + .../datasource/physical_plan/file_stream.rs | 25 +- .../core/src/datasource/physical_plan/json.rs | 301 +++----- .../core/src/datasource/physical_plan/mod.rs | 14 +- .../datasource/physical_plan/parquet/mod.rs | 726 ++++++------------ .../core/src/datasource/schema_adapter.rs | 34 +- datafusion/core/src/datasource/view.rs | 5 +- .../enforce_distribution.rs | 491 ++++++------ .../src/physical_optimizer/enforce_sorting.rs | 278 +++---- .../physical_optimizer/projection_pushdown.rs | 219 +++--- .../replace_with_order_preserving_variants.rs | 122 ++- .../src/physical_optimizer/sanity_checker.rs | 22 +- .../core/src/physical_optimizer/test_utils.rs | 30 +- datafusion/core/src/physical_planner.rs | 11 +- datafusion/core/src/test/mod.rs | 78 +- datafusion/core/src/test_util/parquet.rs | 49 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 29 +- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 42 +- .../core/tests/fuzz_cases/merge_fuzz.rs | 6 +- datafusion/core/tests/fuzz_cases/pruning.rs | 31 +- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 6 +- .../sort_preserving_repartition_fuzz.rs | 18 +- .../core/tests/fuzz_cases/window_fuzz.rs | 26 +- datafusion/core/tests/memory_limit/mod.rs | 52 +- .../core/tests/parquet/custom_reader.rs | 31 +- .../tests/parquet/external_access_plan.rs | 38 +- .../core/tests/parquet/file_statistics.rs | 19 +- datafusion/core/tests/parquet/page_pruning.rs | 27 +- .../core/tests/parquet/schema_coercion.rs | 18 +- datafusion/core/tests/parquet/utils.rs | 21 +- .../{parquet_exec.rs => parquet_config.rs} | 2 +- .../combine_partial_final_agg.rs | 25 +- .../limited_distinct_aggregation.rs | 41 +- .../tests/physical_optimizer/test_util.rs | 18 +- datafusion/core/tests/sql/explain_analyze.rs | 6 +- datafusion/core/tests/sql/joins.rs | 12 +- datafusion/core/tests/sql/path_partition.rs | 15 +- .../user_defined_table_functions.rs | 19 +- datafusion/execution/src/memory_pool/mod.rs | 2 +- .../src/aggregate_statistics.rs | 11 +- datafusion/physical-plan/benches/spm.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 40 +- datafusion/physical-plan/src/display.rs | 2 +- .../physical-plan/src/execution_plan.rs | 10 +- .../physical-plan/src/joins/hash_join.rs | 78 +- .../src/joins/nested_loop_join.rs | 13 +- .../src/joins/sort_merge_join.rs | 14 +- .../physical-plan/src/joins/test_utils.rs | 12 +- datafusion/physical-plan/src/lib.rs | 1 + datafusion/physical-plan/src/memory.rs | 126 ++- .../physical-plan/src/repartition/mod.rs | 39 +- .../physical-plan/src/sorts/partial_sort.rs | 26 +- datafusion/physical-plan/src/sorts/sort.rs | 16 +- .../src/sorts/sort_preserving_merge.rs | 55 +- datafusion/physical-plan/src/source.rs | 139 ++++ datafusion/physical-plan/src/test.rs | 15 +- datafusion/physical-plan/src/union.rs | 15 +- datafusion/physical-plan/src/visitor.rs | 6 +- .../src/windows/bounded_window_agg_exec.rs | 13 +- .../proto/src/physical_plan/from_proto.rs | 3 + datafusion/proto/src/physical_plan/mod.rs | 233 +++--- .../tests/cases/roundtrip_physical_plan.rs | 36 +- .../engines/datafusion_engine/normalize.rs | 4 +- .../test_files/agg_func_substitute.slt | 6 +- .../sqllogictest/test_files/aggregate.slt | 26 +- .../test_files/aggregates_topk.slt | 10 +- .../sqllogictest/test_files/arrow_files.slt | 4 +- datafusion/sqllogictest/test_files/avro.slt | 2 +- datafusion/sqllogictest/test_files/copy.slt | 4 +- .../test_files/count_star_rule.slt | 6 +- .../test_files/create_external_table.slt | 6 +- datafusion/sqllogictest/test_files/cse.slt | 22 +- .../sqllogictest/test_files/csv_files.slt | 2 +- datafusion/sqllogictest/test_files/cte.slt | 42 +- datafusion/sqllogictest/test_files/ddl.slt | 2 +- .../sqllogictest/test_files/dictionary.slt | 6 +- .../sqllogictest/test_files/distinct_on.slt | 2 +- .../sqllogictest/test_files/explain.slt | 62 +- datafusion/sqllogictest/test_files/expr.slt | 2 +- .../sqllogictest/test_files/group_by.slt | 122 +-- datafusion/sqllogictest/test_files/insert.slt | 8 +- .../test_files/insert_to_external.slt | 6 +- datafusion/sqllogictest/test_files/join.slt | 30 +- .../join_disable_repartition_joins.slt | 8 +- datafusion/sqllogictest/test_files/joins.slt | 196 ++--- datafusion/sqllogictest/test_files/json.slt | 4 +- datafusion/sqllogictest/test_files/limit.slt | 10 +- datafusion/sqllogictest/test_files/map.slt | 2 +- .../test_files/monotonic_projection_test.slt | 16 +- .../sqllogictest/test_files/options.slt | 6 +- datafusion/sqllogictest/test_files/order.slt | 42 +- .../sqllogictest/test_files/parquet.slt | 18 +- .../test_files/parquet_filter_pushdown.slt | 14 +- .../test_files/parquet_sorted_statistics.slt | 12 +- .../sqllogictest/test_files/predicates.slt | 14 +- .../sqllogictest/test_files/projection.slt | 4 +- .../sqllogictest/test_files/references.slt | 2 +- datafusion/sqllogictest/test_files/regexp.slt | 4 +- .../sqllogictest/test_files/repartition.slt | 4 +- .../test_files/repartition_scan.slt | 16 +- datafusion/sqllogictest/test_files/scalar.slt | 4 +- datafusion/sqllogictest/test_files/select.slt | 42 +- .../test_files/sort_merge_join.slt | 4 +- datafusion/sqllogictest/test_files/struct.slt | 2 +- .../sqllogictest/test_files/subquery.slt | 20 +- .../sqllogictest/test_files/subquery_sort.slt | 10 +- datafusion/sqllogictest/test_files/topk.slt | 2 +- .../test_files/tpch/plans/q1.slt.part | 2 +- .../test_files/tpch/plans/q10.slt.part | 8 +- .../test_files/tpch/plans/q11.slt.part | 12 +- .../test_files/tpch/plans/q12.slt.part | 4 +- .../test_files/tpch/plans/q13.slt.part | 4 +- .../test_files/tpch/plans/q14.slt.part | 4 +- .../test_files/tpch/plans/q15.slt.part | 6 +- .../test_files/tpch/plans/q16.slt.part | 6 +- .../test_files/tpch/plans/q17.slt.part | 6 +- .../test_files/tpch/plans/q18.slt.part | 8 +- .../test_files/tpch/plans/q19.slt.part | 4 +- .../test_files/tpch/plans/q2.slt.part | 18 +- .../test_files/tpch/plans/q20.slt.part | 10 +- .../test_files/tpch/plans/q21.slt.part | 12 +- .../test_files/tpch/plans/q22.slt.part | 6 +- .../test_files/tpch/plans/q3.slt.part | 6 +- .../test_files/tpch/plans/q4.slt.part | 4 +- .../test_files/tpch/plans/q5.slt.part | 12 +- .../test_files/tpch/plans/q6.slt.part | 2 +- .../test_files/tpch/plans/q7.slt.part | 12 +- .../test_files/tpch/plans/q8.slt.part | 16 +- .../test_files/tpch/plans/q9.slt.part | 12 +- datafusion/sqllogictest/test_files/union.slt | 46 +- datafusion/sqllogictest/test_files/unnest.slt | 4 +- datafusion/sqllogictest/test_files/window.slt | 102 +-- .../substrait/src/physical_plan/consumer.rs | 6 +- .../substrait/src/physical_plan/producer.rs | 169 ++-- .../tests/cases/roundtrip_physical_plan.rs | 6 +- .../building-logical-plans.md | 2 +- docs/source/user-guide/explain-usage.md | 34 +- docs/source/user-guide/sql/explain.md | 4 +- 159 files changed, 3443 insertions(+), 3639 deletions(-) create mode 100644 datafusion/core/src/datasource/data_source.rs rename datafusion/core/tests/{parquet_exec.rs => parquet_config.rs} (93%) create mode 100644 datafusion/physical-plan/src/source.rs diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index 36e68ec4842b..6160d6b07282 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -27,7 +27,6 @@ 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::ExecutionPlan; use datafusion::scalar::ScalarValue; use datafusion_catalog::TableFunctionImpl; @@ -241,11 +240,12 @@ impl TableProvider for ParquetMetadataTable { _filters: &[Expr], _limit: Option, ) -> Result> { - Ok(Arc::new(MemoryExec::try_new( + let source = Arc::new(MemorySourceConfig::try_new( &[vec![self.batch.clone()]], TableProvider::schema(self), projection.cloned(), - )?)) + )?); + Ok(Arc::new(DataSourceExec::new(source))) } } diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 28a3a2f1de09..841dec90f0b9 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -15,17 +15,22 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::collections::{HashMap, HashSet}; +use std::fs::File; +use std::ops::Range; +use std::path::{Path, PathBuf}; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::Arc; + use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; use arrow_schema::SchemaRef; -use async_trait::async_trait; -use bytes::Bytes; use datafusion::catalog::Session; +use datafusion::datasource::data_source::FileSourceConfig; 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, ParquetConfig, }; use datafusion::datasource::TableProvider; use datafusion::execution::object_store::ObjectStoreUrl; @@ -40,24 +45,22 @@ use datafusion::parquet::schema::types::ColumnPath; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_optimizer::pruning::PruningPredicate; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; +use datafusion_common::config::TableParquetOptions; use datafusion_common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; use datafusion_expr::utils::conjunction; use datafusion_expr::{TableProviderFilterPushDown, TableType}; use datafusion_physical_expr::utils::{Guarantee, LiteralGuarantee}; + +use async_trait::async_trait; +use bytes::Bytes; use futures::future::BoxFuture; use futures::FutureExt; use object_store::ObjectStore; -use std::any::Any; -use std::collections::{HashMap, HashSet}; -use std::fs::File; -use std::ops::Range; -use std::path::{Path, PathBuf}; -use std::sync::atomic::{AtomicBool, Ordering}; -use std::sync::Arc; use tempfile::TempDir; use url::Url; @@ -82,8 +85,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 [`ParquetConfig`] +/// 4. Pass a row selection (within a row group) to [`ParquetConfig`] /// /// 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 @@ -110,7 +113,7 @@ use url::Url; /// │ └───────────────────┘ │ ┌────────────────────┐ /// │ ┌───────────────────┐ │ │ │ /// │ │ │◀┼ ─ ─ ┐ │ ParquetExec │ -/// │ └───────────────────┘ │ │ (Parquet Reader) │ +/// │ └───────────────────┘ │ │ │ (Parquet Reader) │ /// │ ... │ └ ─ ─ ─ ─│ │ /// │ ┌───────────────────┐ │ │ ╔═══════════════╗ │ /// │ │ │ │ │ ║ParquetMetadata║ │ @@ -118,13 +121,13 @@ use url::Url; /// │ ╔═══════════════════╗ │ └────────────────────┘ /// │ ║ Thrift metadata ║ │ /// │ ╚═══════════════════╝ │ 1. With cached ParquetMetadata, so -/// └───────────────────────┘ the ParquetExec does not re-read / +/// └───────────────────────┘ the ParquetConfig 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 ParquetConfig to read a `RowSelection` (row ranges) /// which will skip unneeded data pages. This requires that the Parquet file has /// a [Page Index]. /// @@ -210,7 +213,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); @@ -477,7 +480,7 @@ 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 _); @@ -494,14 +497,21 @@ impl TableProvider for IndexTableProvider { 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) + let source_config = Arc::new( + ParquetConfig::new( + Arc::clone(&file_scan_config.file_schema), + // provide the predicate so the DataSourceExec can try and prune + // row groups internally + Some(predicate), + None, + TableParquetOptions::default(), + ) // provide the factory to create parquet reader without re-reading metadata - .with_parquet_file_reader_factory(Arc::new(reader_factory)) - .build_arc()) + .with_parquet_file_reader_factory(Arc::new(reader_factory)), + ); + let source = Arc::new(FileSourceConfig::new(file_scan_config, source_config)); + // Finally, put it all together into a DataSourceExec + Ok(Arc::new(DataSourceExec::new(source))) } /// 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 334e4c83404f..ba18e84b0ec4 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -48,19 +48,13 @@ 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 config = CsvConfig::new(true, b',', b'"').with_terminator(Some(b'#')); - let opener = CsvOpener::new(Arc::new(config), FileCompressionType::UNCOMPRESSED); + let opener = Arc::new(CsvOpener::new( + Arc::new(config), + FileCompressionType::UNCOMPRESSED, + object_store, + )); let testdata = datafusion::test_util::arrow_test_data(); let path = format!("{testdata}/csv/aggregate_test_100.csv"); @@ -125,8 +119,12 @@ async fn json_opener() -> Result<()> { .with_limit(Some(5)) .with_file(PartitionedFile::new(path.to_string(), 10)); - let mut stream = - FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new())?; + 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/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index eeb288beb0df..a39cb0d6418b 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -17,11 +17,13 @@ use std::sync::Arc; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ListingOptions, PartitionedFile}; -use datafusion::datasource::physical_plan::ParquetExec; +use datafusion::datasource::physical_plan::ParquetConfig; 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, }; @@ -95,15 +97,26 @@ 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 + .source_config() + .as_any() + .downcast_ref::() + .is_some() + { + self.file_groups = + Some(file_config.base_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 d6e17764442d..ec31a1bc3c2b 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -24,8 +24,9 @@ use arrow::util::pretty::pretty_format_batches; use arrow_schema::SchemaRef; use async_trait::async_trait; use datafusion::catalog::Session; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use datafusion::datasource::TableProvider; use datafusion::execution::object_store::ObjectStoreUrl; use datafusion::parquet::arrow::arrow_reader::statistics::StatisticsConverter; @@ -33,8 +34,10 @@ use datafusion::parquet::arrow::{ arrow_reader::ParquetRecordBatchReaderBuilder, ArrowWriter, }; use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; +use datafusion_common::config::TableParquetOptions; use datafusion_common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -245,7 +248,7 @@ impl TableProvider for IndexTableProvider { .with_projection(projection.cloned()) .with_limit(limit); - // Transform to the format needed to pass to ParquetExec + // 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,14 @@ impl TableProvider for IndexTableProvider { file_size, )); } - let exec = ParquetExec::builder(file_scan_config) - .with_predicate(predicate) - .build_arc(); - - Ok(exec) + let source_config = Arc::new(ParquetConfig::new( + Arc::clone(&file_scan_config.file_schema), + Some(predicate), + None, + TableParquetOptions::default(), + )); + let source = Arc::new(FileSourceConfig::new(file_scan_config, source_config)); + Ok(Arc::new(DataSourceExec::new(source))) } /// 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 206b7ba9c4be..a37eb60bf458 100644 --- a/datafusion-examples/examples/remote_catalog.rs +++ b/datafusion-examples/examples/remote_catalog.rs @@ -36,7 +36,8 @@ use datafusion::catalog::{SchemaProvider, TableProvider}; use datafusion::common::DataFusionError; use datafusion::common::Result; use datafusion::execution::SendableRecordBatchStream; -use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::memory::MemorySourceConfig; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{DataFrame, SessionContext}; @@ -336,11 +337,12 @@ impl TableProvider for RemoteTable { .await? .try_collect() .await?; - Ok(Arc::new(MemoryExec::try_new( + let source = Arc::new(MemorySourceConfig::try_new( &[batches], self.schema.clone(), projection.cloned(), - )?)) + )?); + Ok(Arc::new(DataSourceExec::new(source))) } } diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index 7cf1ce87690e..47ab92349982 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -24,7 +24,8 @@ use datafusion::catalog::Session; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::execution::context::ExecutionProps; -use datafusion::physical_plan::memory::MemoryExec; +use datafusion::physical_plan::memory::MemorySourceConfig; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; use datafusion_catalog::TableFunctionImpl; @@ -120,11 +121,12 @@ impl TableProvider for LocalCsvTable { } else { self.batches.clone() }; - Ok(Arc::new(MemoryExec::try_new( + let source = Arc::new(MemorySourceConfig::try_new( &[batches], TableProvider::schema(self), projection.cloned(), - )?)) + )?); + Ok(Arc::new(DataSourceExec::new(source))) } } diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index 7d87a37b3b9c..f30a35503447 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -33,10 +33,11 @@ use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMerge use datafusion::physical_plan::{ collect, expressions::{col, PhysicalSortExpr}, - memory::MemoryExec, }; use datafusion::prelude::SessionContext; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; // Initialize the operator using the provided record batches and the sort key // as inputs. All record batches must have the same schema. @@ -55,12 +56,15 @@ fn sort_preserving_merge_operator( }) .collect::(); - let exec = MemoryExec::try_new( - &batches.into_iter().map(|rb| vec![rb]).collect::>(), - schema, - None, - ) - .unwrap(); + let source = Arc::new( + MemorySourceConfig::try_new( + &batches.into_iter().map(|rb| vec![rb]).collect::>(), + schema, + None, + ) + .unwrap(), + ); + let exec = DataSourceExec::new(source); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); let task_ctx = session_ctx.task_ctx(); let rt = Runtime::new().unwrap(); diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 14e80ce364e3..bfa1f7a34001 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -79,17 +79,19 @@ use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::{ execution::context::TaskContext, physical_plan::{ - coalesce_partitions::CoalescePartitionsExec, memory::MemoryExec, + coalesce_partitions::CoalescePartitionsExec, sorts::sort_preserving_merge::SortPreservingMergeExec, ExecutionPlan, ExecutionPlanProperties, }, prelude::SessionContext, }; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; /// 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,7 +169,9 @@ 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 source = + Arc::new(MemorySourceConfig::try_new(partitions, schema, None).unwrap()); + let exec = DataSourceExec::new(source); let plan = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); Self { @@ -186,7 +190,9 @@ 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 source = + Arc::new(MemorySourceConfig::try_new(partitions, schema, None).unwrap()); + let exec = DataSourceExec::new(source); let exec = SortExec::new(sort.clone(), Arc::new(exec)).with_preserve_partitioning(true); let plan = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -208,7 +214,9 @@ 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 source = + Arc::new(MemorySourceConfig::try_new(partitions, schema, None).unwrap()); + let exec = DataSourceExec::new(source); let exec = Arc::new(CoalescePartitionsExec::new(Arc::new(exec))); let plan = Arc::new(SortExec::new(sort, exec)); @@ -229,7 +237,9 @@ 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 source = + Arc::new(MemorySourceConfig::try_new(partitions, schema, None).unwrap()); + let exec = DataSourceExec::new(source); let exec = SortExec::new(sort, Arc::new(exec)).with_preserve_partitioning(true); let plan = Arc::new(CoalescePartitionsExec::new(Arc::new(exec))); diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 60a09301ae0f..0861cf29190a 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2462,7 +2462,7 @@ mod tests { &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; @@ -2506,7 +2506,7 @@ mod tests { "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; @@ -2555,7 +2555,7 @@ mod tests { "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; @@ -2606,7 +2606,7 @@ mod tests { "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/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs new file mode 100644 index 000000000000..54a09c8ce433 --- /dev/null +++ b/datafusion/core/src/datasource/data_source.rs @@ -0,0 +1,364 @@ +// 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 trait implementation + +use std::any::Any; +use std::fmt; +use std::fmt::Formatter; +use std::sync::Arc; + +use crate::datasource::listing::PartitionedFile; +use crate::datasource::physical_plan::{ + ArrowConfig, AvroConfig, CsvConfig, FileGroupPartitioner, FileOpener, FileScanConfig, + FileStream, JsonConfig, ParquetConfig, +}; + +use arrow_schema::SchemaRef; +use datafusion_common::config::ConfigOptions; +use datafusion_common::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, MetricBuilder}; +use datafusion_physical_plan::source::{DataSource, DataSourceExec}; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; + +use itertools::Itertools; +use object_store::ObjectStore; + +/// Common behaviors that every `FileSourceConfig` needs to implement. +/// Being stored as source_config in `DataSourceExec`. +pub trait DataSourceFileConfig: 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; +} + +/// Holds generic file configuration, and common behaviors. +/// Can be initialized with a `FileScanConfig` +/// and a `dyn DataSourceFileConfig` type such as `CsvConfig`, `ParquetConfig`, `AvroConfig`, etc. +#[derive(Clone)] +pub struct FileSourceConfig { + source_config: Arc, + base_config: FileScanConfig, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Statistics, + cache: PlanProperties, +} + +impl DataSource for FileSourceConfig { + fn open( + &self, + partition: usize, + context: Arc, + ) -> datafusion_common::Result { + let object_store = context + .runtime_env() + .object_store(&self.base_config.object_store_url); + + let source_config = self + .source_config + .with_batch_size(context.session_config().batch_size()) + .with_schema(Arc::clone(&self.base_config.file_schema)) + .with_projection(&self.base_config); + + let opener = source_config.create_file_opener( + object_store, + &self.base_config, + partition, + )?; + + let stream = + FileStream::new(&self.base_config, partition, opener, &self.metrics)?; + Ok(Box::pin(stream)) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + self.base_config.fmt_as(t, f)?; + self.fmt_source_config(f)?; + + if let Some(csv_conf) = self.source_config.as_any().downcast_ref::() { + write!(f, ", has_header={}", csv_conf.has_header) + } else if let Some(parquet_conf) = + self.source_config.as_any().downcast_ref::() + { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let predicate_string = parquet_conf + .predicate() + .map(|p| format!(", predicate={p}")) + .unwrap_or_default(); + + let pruning_predicate_string = parquet_conf + .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,) + } + } + } else { + Ok(()) + } + } + + /// Redistribute files across partitions according to their size + /// See comments on [`FileGroupPartitioner`] for more detail. + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + exec: DataSourceExec, + ) -> datafusion_common::Result>> { + if !self.supports_repartition() { + return Ok(None); + } + 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.cache().output_ordering().is_some()) + .repartition_file_groups(&self.base_config.file_groups); + + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + let plan = Arc::new(exec.with_source(Arc::new( + self.clone().with_file_groups(repartitioned_file_groups), + ))); + return Ok(Some(plan)); + } + Ok(None) + } + + fn statistics(&self) -> datafusion_common::Result { + let stats = if let Some(parquet_config) = + self.source_config.as_any().downcast_ref::() + { + // 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 parquet_config.pruning_predicate().is_some() + || parquet_config.page_pruning_predicate().is_some() + || (parquet_config.predicate().is_some() + && parquet_config.pushdown_filters()) + { + self.projected_statistics.clone().to_inexact() + } else { + self.projected_statistics.clone() + } + } else { + self.projected_statistics.clone() + }; + + Ok(stats) + } + + fn with_fetch(&self, limit: Option) -> Option> { + let config = self.base_config.clone().with_limit(limit); + Some(Arc::new(Self { + source_config: Arc::clone(&self.source_config), + base_config: config, + metrics: self.metrics.clone(), + projected_statistics: self.projected_statistics.clone(), + cache: self.cache(), + })) + } + + fn fetch(&self) -> Option { + self.base_config.limit + } + + fn metrics(&self) -> ExecutionPlanMetricsSet { + self.metrics.clone() + } + + fn properties(&self) -> PlanProperties { + self.cache() + } +} + +impl FileSourceConfig { + /// Returns a new [`DataSourceExec`] from file configurations + pub fn new_exec( + base_config: FileScanConfig, + source_config: Arc, + ) -> Arc { + let source = Arc::new(Self::new(base_config, source_config)); + + Arc::new(DataSourceExec::new(source)) + } + + /// Initialize a new `FileSourceConfig` instance with metrics, cache, and statistics. + pub fn new( + base_config: FileScanConfig, + source_config: Arc, + ) -> Self { + let (projected_schema, projected_statistics, projected_output_ordering) = + base_config.project(); + let cache = Self::compute_properties( + Arc::clone(&projected_schema), + &projected_output_ordering, + &base_config, + ); + let mut metrics = ExecutionPlanMetricsSet::new(); + + if let Some(parquet_config) = + source_config.as_any().downcast_ref::() + { + metrics = parquet_config.metrics(); + let _predicate_creation_errors = MetricBuilder::new(&metrics) + .global_counter("num_predicate_creation_errors"); + }; + + Self { + source_config, + base_config, + metrics, + projected_statistics, + cache, + } + } + /// Write the data_type based on source_config + fn fmt_source_config(&self, f: &mut Formatter) -> fmt::Result { + let data_type = if self + .source_config + .as_any() + .downcast_ref::() + .is_some() + { + "avro" + } else if self + .source_config + .as_any() + .downcast_ref::() + .is_some() + { + "arrow" + } else if self + .source_config + .as_any() + .downcast_ref::() + .is_some() + { + "csv" + } else if self + .source_config + .as_any() + .downcast_ref::() + .is_some() + { + "json" + } else if self + .source_config + .as_any() + .downcast_ref::() + .is_some() + { + "parquet" + } else { + "unknown" + }; + write!(f, ", file_type={}", data_type) + } + + /// Returns the base_config + pub fn base_config(&self) -> &FileScanConfig { + &self.base_config + } + + /// Returns the source_config + pub fn source_config(&self) -> &Arc { + &self.source_config + } + + /// Returns the `PlanProperties` of the plan + pub(crate) fn cache(&self) -> PlanProperties { + self.cache.clone() + } + + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + 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 + } + + fn supports_repartition(&self) -> bool { + !(self.base_config.file_compression_type.is_compressed() + || self.base_config.new_lines_in_values + || self + .source_config + .as_any() + .downcast_ref::() + .is_some()) + } +} diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 2697e5621af3..d91c8d2dde20 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -29,9 +29,10 @@ use super::file_compression_type::FileCompressionType; use super::write::demux::start_demuxer_task; use super::write::{create_writer, SharedBuffer}; use super::FileFormatFactory; +use crate::datasource::data_source::FileSourceConfig; use crate::datasource::file_format::FileFormat; use crate::datasource::physical_plan::{ - ArrowExec, FileGroupDisplay, FileScanConfig, FileSinkConfig, + ArrowConfig, FileGroupDisplay, FileScanConfig, FileSinkConfig, }; use crate::error::Result; use crate::execution::context::SessionState; @@ -49,12 +50,13 @@ use datafusion_common::{ 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 datafusion_physical_plan::metrics::MetricsSet; +use datafusion_physical_plan::source::DataSourceExec; 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}; @@ -171,7 +173,9 @@ impl FileFormat for ArrowFormat { conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { - let exec = ArrowExec::new(conf); + let source_config = Arc::new(ArrowConfig {}); + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let exec = DataSourceExec::new(source); Ok(Arc::new(exec)) } diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index f854b9506a64..178b58e24b18 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::data_source::FileSourceConfig; +use crate::datasource::physical_plan::{AvroConfig, FileScanConfig}; +use crate::error::Result; +use crate::execution::context::SessionState; +use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::Statistics; + use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; use async_trait::async_trait; @@ -30,18 +41,9 @@ use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::GetExt; use datafusion_common::DEFAULT_AVRO_EXTENSION; use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::source::DataSourceExec; 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; @@ -150,7 +152,9 @@ impl FileFormat for AvroFormat { conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { - let exec = AvroExec::new(conf); + let source_config = Arc::new(AvroConfig::new()); + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let exec = DataSourceExec::new(source); Ok(Arc::new(exec)) } } diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index e8fb3690efbf..34a3b8645419 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -27,10 +27,11 @@ use super::{ Decoder, DecoderDeserializer, FileFormat, FileFormatFactory, DEFAULT_SCHEMA_INFER_MAX_RECORD, }; +use crate::datasource::data_source::FileSourceConfig; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::BatchSerializer; use crate::datasource::physical_plan::{ - CsvExec, FileGroupDisplay, FileScanConfig, FileSinkConfig, + CsvConfig, FileGroupDisplay, FileScanConfig, FileSinkConfig, }; use crate::error::Result; use crate::execution::context::SessionState; @@ -51,11 +52,12 @@ use datafusion_common::{ use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_plan::metrics::MetricsSet; +use datafusion_physical_plan::source::DataSourceExec; 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}; @@ -409,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 @@ -421,17 +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(); + let source_config = Arc::new( + CsvConfig::new(has_header, self.options.delimiter, self.options.quote) + .with_escape(self.options.escape) + .with_terminator(self.options.terminator) + .with_comment(self.options.comment), + ); + + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let exec = DataSourceExec::new(source); Ok(Arc::new(exec)) } @@ -1209,7 +1212,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, @@ -1218,7 +1221,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) { @@ -1227,7 +1230,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 4bdf336881c9..ffcda5f72c1c 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -29,10 +29,10 @@ use super::{ Decoder, DecoderDeserializer, FileFormat, FileFormatFactory, FileScanConfig, DEFAULT_SCHEMA_INFER_MAX_RECORD, }; +use crate::datasource::data_source::FileSourceConfig; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::BatchSerializer; -use crate::datasource::physical_plan::FileGroupDisplay; -use crate::datasource::physical_plan::{FileSinkConfig, NdJsonExec}; +use crate::datasource::physical_plan::{FileGroupDisplay, FileSinkConfig, JsonConfig}; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::insert::{DataSink, DataSinkExec}; @@ -53,6 +53,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::metrics::MetricsSet; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::ExecutionPlan; use async_trait::async_trait; @@ -245,11 +246,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)); + let source_config = Arc::new(JsonConfig::new()); + conf.file_compression_type = FileCompressionType::from(self.options.compression); + + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let exec = DataSourceExec::new(source); Ok(Arc::new(exec)) } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 8f64bea39df7..0ddb93895abf 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -80,10 +80,12 @@ use tokio::io::{AsyncWrite, AsyncWriteExt}; use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::task::JoinSet; +use crate::datasource::data_source::FileSourceConfig; use crate::datasource::physical_plan::parquet::{ - can_expr_be_pushed_down_with_schemas, ParquetExecBuilder, + can_expr_be_pushed_down_with_schemas, ParquetConfig, }; use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::source::DataSourceExec; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; use object_store::path::Path; @@ -402,22 +404,30 @@ impl FileFormat for ParquetFormat { 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 source_config = Arc::new(ParquetConfig::new( + Arc::clone(&conf.file_schema), + predicate, + metadata_size_hint, + self.options.clone(), + )); + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + + Ok(Arc::new(DataSourceExec::new(source))) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index c1e0bea0b3ff..58d7f35a547e 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,7 +41,9 @@ 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::metrics::MetricsSet; +use datafusion_physical_plan::source::DataSourceExec; use async_trait::async_trait; use futures::StreamExt; @@ -162,7 +163,12 @@ impl MemTable { } } - let exec = MemoryExec::try_new(&data, Arc::clone(&schema), None)?; + let source = Arc::new(MemorySourceConfig::try_new( + &data, + Arc::clone(&schema), + None, + )?); + let exec = DataSourceExec::new(source); if let Some(num_partitions) = output_partitions { let exec = RepartitionExec::try_new( @@ -220,11 +226,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(); @@ -241,10 +247,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 7d3fe9ddd751..6419f2655685 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 4e76b087abb1..b952b832cc4a 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,202 +20,55 @@ use std::any::Any; use std::sync::Arc; -use super::FileGroupPartitioner; -use crate::datasource::listing::PartitionedFile; +use crate::datasource::data_source::DataSourceFileConfig; use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, FileScanConfig, }; 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::Statistics; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::PlanProperties; - use futures::StreamExt; use itertools::Itertools; use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; -/// Execution plan for scanning Arrow data source -#[derive(Debug, Clone)] -pub struct ArrowExec { - base_config: FileScanConfig, - projected_statistics: Statistics, - projected_schema: SchemaRef, - projected_output_ordering: Vec, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - cache: PlanProperties, -} - -impl ArrowExec { - /// Create a new Arrow reader execution plan provided base configurations - pub fn new(base_config: FileScanConfig) -> Self { - let (projected_schema, projected_statistics, projected_output_ordering) = - base_config.project(); - let cache = Self::compute_properties( - Arc::clone(&projected_schema), - &projected_output_ordering, - &base_config, - ); - Self { - 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 - } - - fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) - } +/// Arrow configuration struct that is given to [`DataSourceExec`] +/// Does not hold anything special, since [`FileScanConfig`] is sufficient for arrow +#[derive(Clone, Default)] +pub struct ArrowConfig {} - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - projected_output_ordering: &[LexOrdering], - file_scan_config: &FileScanConfig, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = - EquivalenceProperties::new_with_orderings(schema, projected_output_ordering); - - PlanProperties::new( - eq_properties, - Self::output_partitioning_helper(file_scan_config), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - - 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 - } -} - -impl DisplayAs for ArrowExec { - fn fmt_as( +impl DataSourceFileConfig for ArrowConfig { + fn create_file_opener( &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - write!(f, "ArrowExec: ")?; - self.base_config.fmt_as(t, f) - } -} - -impl ExecutionPlan for ArrowExec { - fn name(&self) -> &'static str { - "ArrowExec" + 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 properties(&self) -> &PlanProperties { - &self.cache + fn with_batch_size(&self, _batch_size: usize) -> Arc { + Arc::new(ArrowConfig::default()) } - fn children(&self) -> Vec<&Arc> { - Vec::new() + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(ArrowConfig::default()) } - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - /// Redistribute files across partitions according to their size - /// 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) - } - - 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)) - } - - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) - } - - fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) - } - - fn fetch(&self) -> Option { - self.base_config.limit - } - - 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(), - })) + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + Arc::new(ArrowConfig::default()) } } +/// The struct that implements `[FileOpener]` 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 fb36179c3cf6..5b3f49e691d6 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -20,208 +20,109 @@ use std::any::Any; 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::DataSourceFileConfig; 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_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; - -/// Execution plan for scanning Avro data source -#[derive(Debug, Clone)] -pub struct AvroExec { - base_config: FileScanConfig, - projected_statistics: Statistics, - projected_schema: SchemaRef, - projected_output_ordering: Vec, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - cache: PlanProperties, + +use object_store::ObjectStore; + +/// AvroConfig holds the extra configuration that is necessary for [`AvroOpener`] +#[derive(Clone, Default)] +pub struct AvroConfig { + schema: Option, + batch_size: Option, + projection: Option>, } -impl AvroExec { - /// Create a new Avro reader execution plan provided base configurations - pub fn new(base_config: FileScanConfig) -> Self { - let (projected_schema, projected_statistics, projected_output_ordering) = - base_config.project(); - let cache = Self::compute_properties( - Arc::clone(&projected_schema), - &projected_output_ordering, - &base_config, - ); - Self { - 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 +impl AvroConfig { + /// Initialize an AvroConfig with default values + pub fn new() -> Self { + Self::default() } - /// 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], - file_scan_config: &FileScanConfig, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - let n_partitions = file_scan_config.file_groups.len(); - - PlanProperties::new( - eq_properties, - Partitioning::UnknownPartitioning(n_partitions), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, + #[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 DisplayAs for AvroExec { - fn fmt_as( +impl DataSourceFileConfig for AvroConfig { + #[cfg(feature = "avro")] + fn create_file_opener( &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - write!(f, "AvroExec: ")?; - self.base_config.fmt_as(t, f) - } -} - -impl ExecutionPlan for AvroExec { - fn name(&self) -> &'static str { - "AvroExec" - } - - 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>, - ) -> Result> { - Ok(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 execute( + fn create_file_opener( &self, + _object_store: Result>, + _base_config: &FileScanConfig, _partition: usize, - _context: Arc, - ) -> Result { - Err(crate::error::DataFusionError::NotImplemented( - "Cannot execute avro plan without avro feature enabled".to_string(), - )) + ) -> Result> { + panic!("Avro feature is not enabled in this build") } - #[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)) - } - - fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) + fn as_any(&self) -> &dyn Any { + self } - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) } - fn fetch(&self) -> Option { - self.base_config.limit + fn with_schema(&self, schema: SchemaRef) -> Arc { + let mut conf = self.clone(); + conf.schema = Some(schema); + Arc::new(conf) } - 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(), - })) + fn with_projection(&self, config: &FileScanConfig) -> Arc { + let mut conf = self.clone(); + conf.projection = config.projected_file_column_names(); + Arc::new(conf) } } #[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; use bytes::Buf; use futures::StreamExt; - use object_store::{GetResultPayload, ObjectStore}; - - pub struct AvroConfig { - pub schema: SchemaRef, - pub batch_size: usize, - pub projection: Option>, - pub object_store: Arc, - } - - impl AvroConfig { - fn open(&self, reader: R) -> Result> { - AvroReader::try_new( - reader, - Arc::clone(&self.schema), - self.batch_size, - self.projection.clone(), - ) - } - } + use object_store::GetResultPayload; pub struct AvroOpener { 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)?; @@ -243,6 +144,7 @@ mod private { mod tests { use super::*; use crate::arrow::datatypes::{DataType, Field, SchemaBuilder}; + use crate::datasource::data_source::FileSourceConfig; use crate::datasource::file_format::{avro::AvroFormat, FileFormat}; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; @@ -250,6 +152,9 @@ mod tests { use crate::scalar::ScalarValue; use crate::test::object_store::local_unpartitioned_file; + use datafusion_physical_plan::source::DataSourceExec; + use datafusion_physical_plan::ExecutionPlan; + use futures::StreamExt; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; @@ -289,19 +194,22 @@ mod tests { .infer_schema(&state, &store, std::slice::from_ref(&meta)) .await?; - let avro_exec = AvroExec::new( - FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) - .with_file(meta.into()) - .with_projection(Some(vec![0, 1, 2])), - ); + let conf = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) + .with_file(meta.into()) + .with_projection(Some(vec![0, 1, 2])); + + let source_config = Arc::new(AvroConfig::new()); + + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let source_exec = DataSourceExec::new(source); 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"); @@ -361,20 +269,22 @@ 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 conf = FileScanConfig::new(object_store_url, file_schema) + .with_file(meta.into()) + .with_projection(projection); + + let source_config = Arc::new(AvroConfig::new()); + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let source_exec = DataSourceExec::new(source); 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"); @@ -431,27 +341,26 @@ 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 conf = 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_config = Arc::new(AvroConfig::new()); + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let source_exec = DataSourceExec::new(source); + 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 a00e74cf4fcd..d9ff46456c1f 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -22,256 +22,85 @@ 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::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::listing::{FileRange, ListingTableUrl}; +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_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use crate::datasource::data_source::DataSourceFileConfig; 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. +/// A Config for [`CsvOpener`] /// -/// # Example: create a `CsvExec` +/// # Example: create a `DataSourceExec` for CSV /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; /// # use datafusion::datasource::{ -/// # physical_plan::{CsvExec, FileScanConfig}, +/// # physical_plan::FileScanConfig, /// # listing::PartitionedFile, /// # }; +/// # use datafusion::datasource::physical_plan::CsvConfig; /// # 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()); -/// // Create a CsvExec for reading the first 100MB of `file1.csv` +/// // Create a DataSourceExec 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(); -/// ``` -#[derive(Debug, Clone)] -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, -} - -/// Builder for [`CsvExec`]. +/// .with_file(PartitionedFile::new("file1.csv", 100*1024*1024)) +/// .with_newlines_in_values(true); // The file contains newlines in values; /// -/// See example on [`CsvExec`]. +/// let source_config = Arc::new(CsvConfig::new( +/// true, +/// b',', +/// b'"', +/// ) +/// .with_terminator(Some(b'#') +/// )); +/// let exec = DataSourceExec::builder(file_scan_config, source_config); +/// ``` #[derive(Debug, Clone)] -pub struct CsvExecBuilder { - file_scan_config: FileScanConfig, - file_compression_type: FileCompressionType, - // TODO: it seems like these format options could be reused across all the various CSV config - has_header: bool, +pub struct CsvConfig { + batch_size: Option, + file_schema: Option, + file_projection: Option>, + pub(crate) has_header: bool, delimiter: u8, quote: u8, terminator: Option, escape: Option, comment: Option, - newlines_in_values: bool, } -impl CsvExecBuilder { - /// Create a new builder to read the provided file scan configuration. - pub fn new(file_scan_config: FileScanConfig) -> Self { +impl CsvConfig { + /// Returns a [`CsvConfig`] + pub fn new(has_header: bool, delimiter: u8, quote: u8) -> Self { Self { - file_scan_config, - // TODO: these defaults are duplicated from `CsvOptions` - should they be computed? - has_header: false, - delimiter: b',', - quote: b'"', - terminator: None, - escape: None, - comment: None, - newlines_in_values: false, - file_compression_type: FileCompressionType::UNCOMPRESSED, - } - } - - /// Set whether the first row defines the column names. - /// - /// The default value is `false`. - pub fn with_has_header(mut self, has_header: bool) -> Self { - self.has_header = has_header; - self - } - - /// Set the column delimeter. - /// - /// The default is `,`. - pub fn with_delimeter(mut self, delimiter: u8) -> Self { - self.delimiter = delimiter; - self - } - - /// Set the quote character. - /// - /// The default is `"`. - pub fn with_quote(mut self, quote: u8) -> Self { - self.quote = quote; - self - } - - /// Set the line terminator. If not set, the default is CRLF. - /// - /// The default is None. - pub fn with_terminator(mut self, terminator: Option) -> Self { - self.terminator = terminator; - self - } - - /// Set the escape character. - /// - /// The default is `None` (i.e. quotes cannot be escaped). - pub fn with_escape(mut self, escape: Option) -> Self { - self.escape = escape; - self - } - - /// Set the comment character. - /// - /// The default is `None` (i.e. comments are not supported). - pub fn with_comment(mut self, comment: Option) -> Self { - self.comment = comment; - self - } - - /// Set 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 value is `false`. - pub fn with_newlines_in_values(mut self, newlines_in_values: bool) -> Self { - self.newlines_in_values = newlines_in_values; - self - } - - /// Set the file compression type. - /// - /// The default is [`FileCompressionType::UNCOMPRESSED`]. - pub fn with_file_compression_type( - mut self, - file_compression_type: FileCompressionType, - ) -> Self { - self.file_compression_type = file_compression_type; - self - } - - /// Build a [`CsvExec`]. - #[must_use] - pub fn build(self) -> CsvExec { - let Self { - file_scan_config: base_config, - file_compression_type, - has_header, - delimiter, - quote, - terminator, - escape, - comment, - newlines_in_values, - } = self; - - let (projected_schema, projected_statistics, projected_output_ordering) = - base_config.project(); - let cache = CsvExec::compute_properties( - projected_schema, - &projected_output_ordering, - &base_config, - ); - - CsvExec { - base_config, - projected_statistics, + batch_size: None, + file_schema: None, + file_projection: None, has_header, delimiter, quote, - terminator, - escape, - newlines_in_values, - metrics: ExecutionPlanMetricsSet::new(), - file_compression_type, - cache, - comment, + terminator: None, + escape: None, + comment: None, } } -} - -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, - has_header: bool, - delimiter: u8, - quote: u8, - terminator: Option, - escape: Option, - comment: Option, - newlines_in_values: bool, - file_compression_type: FileCompressionType, - ) -> Self { - CsvExecBuilder::new(base_config) - .with_has_header(has_header) - .with_delimeter(delimiter) - .with_quote(quote) - .with_terminator(terminator) - .with_escape(escape) - .with_comment(comment) - .with_newlines_in_values(newlines_in_values) - .with_file_compression_type(file_compression_type) - .build() - } - - /// Return a [`CsvExecBuilder`]. - /// - /// See example on [`CsvExec`] and [`CsvExecBuilder`] for specifying CSV table options. - pub fn builder(file_scan_config: FileScanConfig) -> CsvExecBuilder { - CsvExecBuilder::new(file_scan_config) - } - /// Ref to the base configs - 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 @@ -301,219 +130,25 @@ impl CsvExec { self.escape } - /// 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.newlines_in_values - } - - fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) - } - - /// 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], - file_scan_config: &FileScanConfig, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - - PlanProperties::new( - eq_properties, - Self::output_partitioning_helper(file_scan_config), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - - 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 - } -} - -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) - } -} - -impl ExecutionPlan for CsvExec { - fn name(&self) -> &'static str { - "CsvExec" - } - - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - &self.cache - } - - fn children(&self) -> Vec<&Arc> { - // this is a leaf node and has no children - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - /// Redistribute files across partitions according to their size - /// See comments on [`FileGroupPartitioner`] for more detail. - /// - /// Return `None` if can't get repartitioned (empty, compressed file, or `newlines_in_values` set). - fn repartitioned( - &self, - 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) - } - - fn execute( - &self, - 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) - } - - fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) - } - - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) - } - - fn fetch(&self) -> Option { - self.base_config.limit + /// Initialize a CsvConfig with escape + pub fn with_escape(&self, escape: Option) -> Self { + let mut conf = self.clone(); + conf.escape = escape; + conf } - 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(), - })) + /// Initialize a CsvConfig with terminator + pub fn with_terminator(&self, terminator: Option) -> Self { + let mut conf = self.clone(); + conf.terminator = terminator; + conf } -} - -/// A Config for [`CsvOpener`] -#[derive(Debug, Clone)] -pub struct CsvConfig { - batch_size: usize, - file_schema: SchemaRef, - file_projection: Option>, - has_header: bool, - delimiter: u8, - quote: u8, - terminator: Option, - escape: Option, - object_store: Arc, - comment: 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 { - Self { - batch_size, - file_schema, - file_projection, - has_header, - delimiter, - quote, - terminator, - escape: None, - object_store, - comment, - } + /// Initialize a CsvConfig with comment + pub fn with_comment(&self, comment: Option) -> Self { + let mut conf = self.clone(); + conf.comment = comment; + conf } } @@ -523,11 +158,18 @@ impl CsvConfig { } 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); } @@ -549,6 +191,7 @@ impl CsvConfig { pub struct CsvOpener { config: Arc, file_compression_type: FileCompressionType, + object_store: Arc, } impl CsvOpener { @@ -556,14 +199,53 @@ impl CsvOpener { pub fn new( config: Arc, file_compression_type: FileCompressionType, + object_store: Arc, ) -> Self { Self { config, file_compression_type, + object_store, } } } +impl DataSourceFileConfig for CsvConfig { + 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_projection(&self, config: &FileScanConfig) -> Arc { + let mut conf = self.clone(); + conf.file_projection = config.file_column_projection_indices(); + Arc::new(conf) + } +} + impl FileOpener for CsvOpener { /// Open a partitioned CSV file. /// @@ -613,7 +295,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 { @@ -738,8 +420,9 @@ mod tests { use bytes::Bytes; use datafusion_common::test_util::arrow_test_data; - use datafusion_common::config::CsvOptions; - use datafusion_execution::object_store::ObjectStoreUrl; + use crate::datasource::data_source::FileSourceConfig; + use datafusion_physical_plan::metrics::MetricsSet; + use datafusion_physical_plan::source::DataSourceExec; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; @@ -777,20 +460,16 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); + let mut config = partitioned_csv_config(file_schema, file_groups) + .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 source_config = Arc::new(CsvConfig::new(true, b',', b'"')); + let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); + let csv = Arc::new(DataSourceExec::new(source)); + + assert_eq!(13, config.file_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); let mut stream = csv.execute(0, task_ctx)?; @@ -847,20 +526,15 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); + let mut config = partitioned_csv_config(file_schema, file_groups) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()); 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) - .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 source_config = Arc::new(CsvConfig::new(true, b',', b'"')); + let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); + let csv = Arc::new(DataSourceExec::new(source)); + assert_eq!(13, config.file_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); let mut stream = csv.execute(0, task_ctx)?; @@ -917,20 +591,15 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); + let mut config = partitioned_csv_config(file_schema, file_groups) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()); 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) - .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 source_config = Arc::new(CsvConfig::new(true, b',', b'"')); + let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); + let csv = Arc::new(DataSourceExec::new(source)); + assert_eq!(13, config.file_schema.fields().len()); assert_eq!(13, csv.schema().fields().len()); let mut it = csv.execute(0, task_ctx)?; @@ -984,20 +653,15 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); + let mut config = partitioned_csv_config(file_schema, file_groups) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()); 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) - .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()); + let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); + let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); + let csv = Arc::new(DataSourceExec::new(source)); + 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 @@ -1041,7 +705,9 @@ mod tests { tmp_dir.path(), )?; - let mut config = partitioned_csv_config(file_schema, file_groups); + let mut config = partitioned_csv_config(file_schema, file_groups) + .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)]; @@ -1053,17 +719,11 @@ 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 source_config = Arc::new(CsvConfig::new(true, b',', b'"')); + let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); + let csv = Arc::new(DataSourceExec::new(source)); + assert_eq!(13, config.file_schema.fields().len()); assert_eq!(2, csv.schema().fields().len()); let mut it = csv.execute(0, task_ctx)?; @@ -1148,17 +808,12 @@ 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 config = partitioned_csv_config(file_schema, file_groups) .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 source_config = Arc::new(CsvConfig::new(true, b',', b'"')); + let source = Arc::new(FileSourceConfig::new(config, source_config)); + let csv = Arc::new(DataSourceExec::new(source)); let it = csv.execute(0, task_ctx).unwrap(); let batches: Vec<_> = it.try_collect().await.unwrap(); @@ -1467,36 +1122,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 a5f2bd1760b3..540d5280fd57 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -24,6 +24,7 @@ use std::{ }; 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}; @@ -127,6 +128,10 @@ 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, + /// New Lines in Values for CSVOptions + pub new_lines_in_values: bool, } impl FileScanConfig { @@ -151,6 +156,8 @@ impl FileScanConfig { limit: None, table_partition_cols: vec![], output_ordering: vec![], + file_compression_type: FileCompressionType::UNCOMPRESSED, + new_lines_in_values: false, } } @@ -210,6 +217,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 and the statistics on the given column indices pub fn project(&self) -> (SchemaRef, Statistics, Vec) { if self.projection.is_none() && self.table_partition_cols.is_empty() { diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 18cda4524ab2..2d8884c5eb59 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) } @@ -652,9 +652,10 @@ mod tests { .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 4071f9c26b58..70a872d8b849 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -22,237 +22,97 @@ 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::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::listing::ListingTableUrl; +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_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use crate::datasource::data_source::DataSourceFileConfig; 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 NdJson data source -#[derive(Debug, Clone)] -pub struct NdJsonExec { - base_config: FileScanConfig, - projected_statistics: Statistics, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, +/// A [`FileOpener`] that opens a JSON file and yields a [`FileOpenFuture`] +pub struct JsonOpener { + batch_size: usize, + projected_schema: SchemaRef, file_compression_type: FileCompressionType, - cache: PlanProperties, + object_store: Arc, } -impl NdJsonExec { - /// Create a new JSON reader execution plan provided base configurations +impl JsonOpener { + /// Returns a [`JsonOpener`] pub fn new( - base_config: FileScanConfig, + batch_size: usize, + projected_schema: SchemaRef, file_compression_type: FileCompressionType, + object_store: Arc, ) -> Self { - let (projected_schema, projected_statistics, projected_output_ordering) = - base_config.project(); - let cache = Self::compute_properties( - projected_schema, - &projected_output_ordering, - &base_config, - ); Self { - base_config, - projected_statistics, - metrics: ExecutionPlanMetricsSet::new(), + batch_size, + projected_schema, file_compression_type, - cache, + object_store, } } +} - /// Ref to the base configs - pub fn base_config(&self) -> &FileScanConfig { - &self.base_config - } - - /// Ref to file compression type - pub fn file_compression_type(&self) -> &FileCompressionType { - &self.file_compression_type - } - - fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) - } - - /// 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], - file_scan_config: &FileScanConfig, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - - PlanProperties::new( - eq_properties, - Self::output_partitioning_helper(file_scan_config), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - - 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 - } +/// JsonConfig holds the extra configuration that is necessary for [`JsonOpener`] +#[derive(Clone, Default)] +pub struct JsonConfig { + batch_size: Option, } -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) +impl JsonConfig { + /// Initialize a JsonConfig with default values + pub fn new() -> Self { + Self::default() } } -impl ExecutionPlan for NdJsonExec { - fn name(&self) -> &'static str { - "NdJsonExec" +impl DataSourceFileConfig for JsonConfig { + 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 properties(&self) -> &PlanProperties { - &self.cache - } - - fn children(&self) -> Vec<&Arc> { - Vec::new() - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - fn repartitioned( - &self, - 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) - } - - fn execute( - &self, - 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) + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) } - fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(Self { ..*self }) } - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) - } - - fn fetch(&self) -> Option { - self.base_config.limit - } - - 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(), - })) - } -} - -/// A [`FileOpener`] that opens a JSON file and yields a [`FileOpenFuture`] -pub struct JsonOpener { - batch_size: usize, - projected_schema: SchemaRef, - file_compression_type: FileCompressionType, - object_store: Arc, -} - -impl JsonOpener { - /// Returns a [`JsonOpener`] - pub fn new( - batch_size: usize, - projected_schema: SchemaRef, - file_compression_type: FileCompressionType, - object_store: Arc, - ) -> Self { - Self { - batch_size, - projected_schema, - file_compression_type, - object_store, - } + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + Arc::new(Self { ..*self }) } } @@ -385,7 +245,9 @@ mod tests { use super::*; use crate::dataframe::DataFrameWriteOptions; + use crate::datasource::data_source::FileSourceConfig; 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::{ @@ -397,6 +259,8 @@ 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 datafusion_physical_plan::source::DataSourceExec; + use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; @@ -519,12 +383,15 @@ 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 conf = FileScanConfig::new(object_store_url, file_schema) + .with_file_groups(file_groups) + .with_limit(Some(3)) + .with_file_compression_type(file_compression_type.to_owned()); + + let source_config = Arc::new(JsonConfig::new()); + + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let exec = DataSourceExec::new(source); // TODO: this is not where schema inference should be tested @@ -590,12 +457,14 @@ 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 conf = FileScanConfig::new(object_store_url, file_schema) + .with_file_groups(file_groups) + .with_limit(Some(3)) + .with_file_compression_type(file_compression_type.to_owned()); + + let source_config = Arc::new(JsonConfig::new()); + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let exec = DataSourceExec::new(source); let mut it = exec.execute(0, task_ctx)?; let batch = it.next().await.unwrap()?; @@ -630,12 +499,14 @@ 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 conf = FileScanConfig::new(object_store_url, file_schema) + .with_file_groups(file_groups) + .with_projection(Some(vec![0, 2])) + .with_file_compression_type(file_compression_type.to_owned()); + + let source_config = Arc::new(JsonConfig::new()); + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let exec = DataSourceExec::new(source); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 2); @@ -675,12 +546,14 @@ 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 conf = FileScanConfig::new(object_store_url, file_schema) + .with_file_groups(file_groups) + .with_projection(Some(vec![3, 0, 2])) + .with_file_compression_type(file_compression_type.to_owned()); + + let source_config = Arc::new(JsonConfig::new()); + let source = Arc::new(FileSourceConfig::new(conf, source_config)); + let exec = DataSourceExec::new(source); 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 3146d124d9f1..d1b84a5a8cf7 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -31,18 +31,18 @@ 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::{ParquetConfig, ParquetFileMetrics, ParquetFileReaderFactory}; -pub use arrow_file::ArrowExec; -pub use avro::AvroExec; -pub use csv::{CsvConfig, CsvExec, CsvExecBuilder, CsvOpener}; +pub use arrow_file::ArrowConfig; +pub use avro::AvroConfig; +pub use csv::{CsvConfig, CsvOpener}; 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}; +pub use json::{JsonConfig, JsonOpener}; use std::{ fmt::{Debug, Formatter, Result as FmtResult}, @@ -301,7 +301,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 @@ -327,7 +327,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/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 83b544a76e11..1cb81c76d2c2 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -18,33 +18,25 @@ //! [`ParquetExec`] Execution plan for reading Parquet files use std::any::Any; -use std::fmt::Debug; use std::sync::Arc; -use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::file_stream::FileStream; +use crate::datasource::data_source::DataSourceFileConfig; use crate::datasource::physical_plan::{ - parquet::page_filter::PagePruningAccessPlanFilter, DisplayAs, FileGroupPartitioner, - FileScanConfig, + parquet::page_filter::PagePruningAccessPlanFilter, FileOpener, FileScanConfig, }; use crate::{ - config::{ConfigOptions, TableParquetOptions}, + config::TableParquetOptions, error::Result, - execution::context::TaskContext, physical_optimizer::pruning::PruningPredicate, - physical_plan::{ - metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, Statistics, - }, + physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}, }; use arrow::datatypes::SchemaRef; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use arrow_schema::Schema; +use datafusion_physical_expr::PhysicalExpr; -use itertools::Itertools; use log::debug; +use object_store::ObjectStore; mod access_plan; mod metrics; @@ -75,7 +67,7 @@ pub use writer::plan_to_parquet; /// │ /// ┌───────────────────────┐ /// │ │ -/// │ ParquetExec │ +/// │ DataSourceExec │ /// │ │ /// └───────────────────────┘ /// ▲ @@ -94,24 +86,33 @@ pub use writer::plan_to_parquet; /// /// ``` /// -/// # Example: Create a `ParquetExec` +/// # Example: Create a `DataSourceExec` /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +/// # use datafusion::datasource::data_source::FileSourceConfig; +/// # use datafusion::datasource::physical_plan::{FileScanConfig, ParquetConfig}; /// # 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; +/// # 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); -/// // Create a ParquetExec for reading `file1.parquet` with a file size of 100MB +/// // Create a DataSourceExec 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(); +/// let source_config = Arc::new( +/// ParquetConfig::new( +/// Arc::clone(&file_scan_config.file_schema), +/// Some(predicate), +/// None, +/// TableParquetOptions::default() +/// ) +/// ); +/// let exec = DataSourceExec::new(Arc::new(FileSourceConfig::new(file_scan_config, source_config))); /// ``` /// /// # Features @@ -139,14 +140,14 @@ pub use writer::plan_to_parquet; /// * 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. +/// is ignored. [`ParquetConfig::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 +/// `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 @@ -167,29 +168,33 @@ pub use writer::plan_to_parquet; /// [`RowFilter`]: parquet::arrow::arrow_reader::RowFilter /// [Parquet PageIndex]: https://github.com/apache/parquet-format/blob/master/PageIndex.md /// -/// # Example: rewriting `ParquetExec` +/// # Example: rewriting `DataSourceExec` /// -/// You can modify a `ParquetExec` using [`ParquetExecBuilder`], for example +/// You can modify a `DataSourceExec` using [`ParquetConfig`], 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::data_source::FileSourceConfig; +/// # use datafusion::datasource::physical_plan::{FileScanConfig}; /// # use datafusion::datasource::listing::PartitionedFile; -/// # fn parquet_exec() -> ParquetExec { unimplemented!() } -/// // Split a single ParquetExec into multiple ParquetExecs, one for each file +/// # 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 existing_file_groups = &exec.base_config().file_groups; +/// let data_source = exec.source(); +/// let existing_file_groups = &data_source.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 +/// // create a new exec by copying the existing exec's source config +/// let new_config = data_source +/// .base_config() +/// .with_file_groups(vec![file_group.clone()]); +/// +/// DataSourceExec::new(Arc::new(FileSourceConfig::new(new_config, data_source.source_config()))) /// }) /// .collect::>(); /// ``` @@ -211,9 +216,12 @@ pub use writer::plan_to_parquet; /// # use std::sync::Arc; /// # use arrow_schema::{Schema, SchemaRef}; /// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion::datasource::data_source::FileSourceConfig; /// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; -/// # use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +/// # use datafusion::datasource::physical_plan::{FileScanConfig, ParquetConfig}; /// # use datafusion_execution::object_store::ObjectStoreUrl; +/// # use datafusion_physical_plan::source::DataSourceExec; +/// /// # fn schema() -> SchemaRef { /// # Arc::new(Schema::empty()) /// # } @@ -227,9 +235,11 @@ pub use writer::plan_to_parquet; /// // create a ParquetExec to scan this file /// let file_scan_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema()) /// .with_file(partitioned_file); +/// // create a ParguetConfig for file opener configurations +/// let source_config = Arc::new(ParquetConfig::default()); /// // 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(); +/// let exec = DataSourceExec::new(Arc::new(FileSourceConfig::new(file_scan_config, source_config))); /// ``` /// /// For a complete example, see the [`advanced_parquet_index` example]). @@ -238,7 +248,7 @@ pub use writer::plan_to_parquet; /// /// # Execution Overview /// -/// * Step 1: [`ParquetExec::execute`] is called, returning a [`FileStream`] +/// * 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 @@ -260,12 +270,11 @@ pub use writer::plan_to_parquet; /// [`RecordBatch`]: arrow::record_batch::RecordBatch /// [`SchemaAdapter`]: crate::datasource::schema_adapter::SchemaAdapter /// [`ParquetMetadata`]: parquet::file::metadata::ParquetMetaData -#[derive(Debug, Clone)] -pub struct ParquetExec { - /// Base configuration for this scan - base_config: FileScanConfig, - projected_statistics: Statistics, - /// Execution metrics +#[derive(Clone, Default, Debug)] +pub struct ParquetConfig { + /// Options for reading Parquet files + table_parquet_options: TableParquetOptions, + /// Optional metrics metrics: ExecutionPlanMetricsSet, /// Optional predicate for row filtering during parquet scan predicate: Option>, @@ -273,160 +282,46 @@ pub struct ParquetExec { 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>, -} - -impl From for ParquetExecBuilder { - fn from(exec: ParquetExec) -> Self { - exec.into_builder() - } -} - -/// [`ParquetExecBuilder`], builder for [`ParquetExec`]. -/// -/// See example on [`ParquetExec`]. -pub struct ParquetExecBuilder { - file_scan_config: FileScanConfig, - predicate: Option>, + /// Batch size configuration + batch_size: Option, + /// Optional hint for the size of the parquet metadata metadata_size_hint: Option, - table_parquet_options: TableParquetOptions, - parquet_file_reader_factory: Option>, - schema_adapter_factory: Option>, } -impl ParquetExecBuilder { - /// Create a new builder to read the provided file scan configuration - pub fn new(file_scan_config: FileScanConfig) -> Self { - Self::new_with_options(file_scan_config, TableParquetOptions::default()) - } - - /// Create a new builder to read the data specified in the file scan - /// configuration with the provided `TableParquetOptions`. - pub fn new_with_options( - file_scan_config: FileScanConfig, - table_parquet_options: TableParquetOptions, - ) -> Self { - Self { - file_scan_config, - predicate: None, - metadata_size_hint: None, - table_parquet_options, - parquet_file_reader_factory: None, - schema_adapter_factory: None, - } - } - - /// Update the list of files groups to read - pub fn with_file_groups(mut self, file_groups: Vec>) -> Self { - self.file_scan_config.file_groups = file_groups; - self - } - - /// Set the filter predicate when reading. - /// - /// See the "Predicate Pushdown" section of the [`ParquetExec`] documentation - /// for more details. - pub fn with_predicate(mut self, predicate: Arc) -> Self { - self.predicate = Some(predicate); - self - } - - /// 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 ParquetExec 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 - } - - /// Set the options for controlling how the ParquetExec reads parquet files. - /// - /// See also [`Self::new_with_options`] - pub fn with_table_parquet_options( - mut self, +impl ParquetConfig { + /// Initialize a ParquetConfig, if default values are going to be used, + /// use `ParguetConfig::default()` instead + pub fn new( + file_schema: Arc, + predicate: Option>, + metadata_size_hint: Option, table_parquet_options: TableParquetOptions, ) -> Self { - self.table_parquet_options = table_parquet_options; - self - } - - /// Set optional user defined parquet file reader factory. - /// - /// You can use [`ParquetFileReaderFactory`] to more precisely control how - /// data is read from parquet files (e.g. skip re-reading metadata, coalesce - /// I/O operations, etc). - /// - /// The default reader factory reads directly from an [`ObjectStore`] - /// instance using individual I/O operations for the footer and each page. - /// - /// If a custom `ParquetFileReaderFactory` is provided, then data access - /// operations will be routed to this factory instead of [`ObjectStore`]. - /// - /// [`ObjectStore`]: object_store::ObjectStore - 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 - } - - /// 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 - } - - /// Convenience: build an `Arc`d `ParquetExec` from this builder - pub fn build_arc(self) -> Arc { - Arc::new(self.build()) - } - - /// Build a [`ParquetExec`] - #[must_use] - pub fn build(self) -> ParquetExec { - let Self { - file_scan_config, - predicate, - metadata_size_hint, - table_parquet_options, - parquet_file_reader_factory, - schema_adapter_factory, - } = self; - - let base_config = file_scan_config; - debug!("Creating ParquetExec, files: {:?}, projection {:?}, predicate: {:?}, limit: {:?}", - base_config.file_groups, base_config.projection, predicate, base_config.limit); + debug!("Creating ParquetConfig, schema: {:?}, predicate: {:?}, metadata_size_hint: {:?}", file_schema, predicate, metadata_size_hint); 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 mut conf = ParquetConfig::new_with_options(table_parquet_options); + conf.with_metrics(metrics); + if let Some(predicate) = predicate.clone() { + conf = conf.with_predicate(predicate); + } + + if let Some(metadata_size_hint) = metadata_size_hint { + conf = conf.with_metadata_size_hint(metadata_size_hint); + } + let pruning_predicate = predicate .clone() .and_then(|predicate_expr| { - match PruningPredicate::try_new(predicate_expr, Arc::clone(file_schema)) { + 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}"); @@ -436,100 +331,68 @@ impl ParquetExecBuilder { } }) .filter(|p| !p.always_true()); + if let Some(pruning_predicate) = pruning_predicate { + conf = conf.with_pruning_predicate(pruning_predicate); + } let page_pruning_predicate = predicate .as_ref() .map(|predicate_expr| { - PagePruningAccessPlanFilter::new(predicate_expr, Arc::clone(file_schema)) + PagePruningAccessPlanFilter::new(predicate_expr, Arc::clone(&file_schema)) }) .map(Arc::new); + if let Some(page_pruning_predicate) = page_pruning_predicate { + conf = conf.with_page_pruning_predicate(page_pruning_predicate); + } - let (projected_schema, projected_statistics, projected_output_ordering) = - base_config.project(); + conf + } - let cache = ParquetExec::compute_properties( - projected_schema, - &projected_output_ordering, - &base_config, - ); - ParquetExec { - base_config, - projected_statistics, - metrics, - predicate, - pruning_predicate, - page_pruning_predicate, - metadata_size_hint, - parquet_file_reader_factory, - cache, + /// Create a new builder to read the data specified in the file scan + /// configuration with the provided `TableParquetOptions`. + pub fn new_with_options(table_parquet_options: TableParquetOptions) -> Self { + Self { table_parquet_options, - schema_adapter_factory, + ..Self::default() } } -} -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>, - metadata_size_hint: Option, - table_parquet_options: TableParquetOptions, - ) -> Self { - let mut builder = - ParquetExecBuilder::new_with_options(base_config, table_parquet_options); - if let Some(predicate) = predicate { - builder = builder.with_predicate(predicate); - } - if let Some(metadata_size_hint) = metadata_size_hint { - builder = builder.with_metadata_size_hint(metadata_size_hint); - } - builder.build() + /// 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 ParquetExec 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 } - /// Return a [`ParquetExecBuilder`]. - /// - /// See example on [`ParquetExec`] and [`ParquetExecBuilder`] for specifying - /// parquet table options. - pub fn builder(file_scan_config: FileScanConfig) -> ParquetExecBuilder { - ParquetExecBuilder::new(file_scan_config) - } - - /// Convert this `ParquetExec` into a builder for modification - pub fn into_builder(self) -> ParquetExecBuilder { - // 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; - ParquetExecBuilder { - file_scan_config: base_config, - predicate, - metadata_size_hint, - table_parquet_options, - parquet_file_reader_factory, - schema_adapter_factory, - } + fn with_metrics(&self, metrics: ExecutionPlanMetricsSet) -> Self { + let mut conf = self.clone(); + conf.metrics = metrics; + conf } - /// [`FileScanConfig`] that controls this scan (such as which files to read) - pub fn base_config(&self) -> &FileScanConfig { - &self.base_config + fn with_predicate(&self, predicate: Arc) -> Self { + let mut conf = self.clone(); + conf.predicate = Some(predicate); + conf + } + + fn with_pruning_predicate(&self, pruning_predicate: Arc) -> Self { + let mut conf = self.clone(); + conf.pruning_predicate = Some(pruning_predicate); + conf + } + + fn with_page_pruning_predicate( + &self, + page_pruning_predicate: Arc, + ) -> Self { + let mut conf = self.clone(); + conf.page_pruning_predicate = Some(page_pruning_predicate); + conf } /// Options passed to the parquet reader for this scan @@ -547,6 +410,11 @@ impl ParquetExec { 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, @@ -569,9 +437,12 @@ impl ParquetExec { 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, @@ -590,10 +461,15 @@ impl ParquetExec { } /// Return the value described in [`Self::with_pushdown_filters`] - fn pushdown_filters(&self) -> bool { + pub(crate) fn pushdown_filters(&self) -> bool { self.table_parquet_options.global.pushdown_filters } + /// Return metrics + pub(crate) fn metrics(&self) -> ExecutionPlanMetricsSet { + self.metrics.clone() + } + /// 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 @@ -644,224 +520,71 @@ impl ParquetExec { fn bloom_filter_on_read(&self) -> bool { self.table_parquet_options.global.bloom_filter_on_read } - - fn output_partitioning_helper(file_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_config.file_groups.len()) - } - - /// 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], - file_config: &FileScanConfig, - ) -> PlanProperties { - PlanProperties::new( - EquivalenceProperties::new_with_orderings(schema, orderings), - Self::output_partitioning_helper(file_config), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - - /// Updates the file groups to read and recalculates the output partitioning - /// - /// Note this function does not update statistics or other properties - /// that depend on the file groups. - fn with_file_groups_and_update_partitioning( - 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 - } } -impl DisplayAs for ParquetExec { - fn fmt_as( +impl DataSourceFileConfig for ParquetConfig { + fn create_file_opener( &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,) - } - } - } -} - -impl ExecutionPlan for ParquetExec { - fn name(&self) -> &'static str { - "ParquetExec" - } - - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - &self.cache - } - - fn children(&self) -> Vec<&Arc> { - // this is a leaf node and has no children - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - /// Redistribute files across partitions according to their size - /// 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))) - } - - fn execute( - &self, - partition_index: usize, - ctx: Arc, - ) -> Result { - let projection = self - .base_config + object_store: Result>, + base_config: &FileScanConfig, + partition: usize, + ) -> Result> { + let projection = base_config .file_column_projection_indices() - .unwrap_or_else(|| { - (0..self.base_config.file_schema.fields().len()).collect() - }); + .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(|| { - ctx.runtime_env() - .object_store(&self.base_config.object_store_url) - .map(|store| { - Arc::new(DefaultParquetFileReaderFactory::new(store)) as _ - }) + object_store.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, + Ok(Arc::new(ParquetOpener { + partition_index: partition, projection: Arc::from(projection), - batch_size: ctx.session_config().batch_size(), - limit: self.base_config.limit, + 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(&self.base_config.file_schema), + table_schema: Arc::clone(&base_config.file_schema), metadata_size_hint: self.metadata_size_hint, - metrics: self.metrics.clone(), + metrics: self.metrics(), 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)) + })) } - fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + fn as_any(&self) -> &dyn Any { + self } - 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) + fn with_batch_size(&self, batch_size: usize) -> Arc { + let mut conf = self.clone(); + conf.batch_size = Some(batch_size); + Arc::new(conf) } - fn fetch(&self) -> Option { - self.base_config.limit + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(Self { ..self.clone() }) } - 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(), - })) + fn with_projection(&self, _config: &FileScanConfig) -> Arc { + Arc::new(Self { ..self.clone() }) } } @@ -886,10 +609,11 @@ mod tests { use super::*; use crate::dataframe::DataFrameWriteOptions; + use crate::datasource::data_source::FileSourceConfig; 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; @@ -912,7 +636,9 @@ 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::MetricsSet; + use datafusion_physical_plan::source::DataSourceExec; + use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use chrono::{TimeZone, Utc}; use futures::StreamExt; @@ -928,7 +654,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 configuration that is used in plan + parquet_config: ParquetConfig, } /// round-trip record batches by writing each individual RecordBatch to @@ -1009,34 +737,39 @@ 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( + let base_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file_group(file_group) - .with_projection(projection), - ); + .with_projection(projection); - if let Some(predicate) = predicate { - builder = builder.with_predicate(predicate); - } - let mut parquet_exec = builder.build(); + let mut source_config = ParquetConfig::new( + Arc::clone(&base_config.file_schema), + predicate, + None, + TableParquetOptions::default(), + ); if pushdown_predicate { - parquet_exec = parquet_exec + source_config = source_config .with_pushdown_filters(true) .with_reorder_filters(true); } if page_index_predicate { - parquet_exec = parquet_exec.with_enable_page_index(true); + source_config = source_config.with_enable_page_index(true); } let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let parquet_exec = Arc::new(parquet_exec); + let source = Arc::new(FileSourceConfig::new( + base_config, + Arc::new(source_config.clone()), + )); + let parquet_exec = Arc::new(DataSourceExec::new(source)); RoundTripResult { batches: collect(parquet_exec.clone(), task_ctx).await, parquet_exec, + parquet_config: source_config, } } } @@ -1667,11 +1400,12 @@ mod tests { expected_row_num: Option, file_schema: SchemaRef, ) -> Result<()> { - let parquet_exec = ParquetExec::builder( + let source = Arc::new(FileSourceConfig::new( FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file_groups(file_groups), - ) - .build(); + Arc::new(ParquetConfig::default()), + )); + let parquet_exec = DataSourceExec::new(source); assert_eq!( parquet_exec .properties() @@ -1768,7 +1502,8 @@ mod tests { ), ]); - let parquet_exec = ParquetExec::builder( + let source_config = Arc::new(ParquetConfig::default()); + let parquet_exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( 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 @@ -1785,12 +1520,14 @@ mod tests { false, ), ]), - ) - .build(); - assert_eq!( - parquet_exec.cache.output_partitioning().partition_count(), - 1 - ); + source_config, + ))); + let partition_count = parquet_exec + .source() + .properties() + .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)?; @@ -1842,11 +1579,11 @@ mod tests { }; let file_schema = Arc::new(Schema::empty()); - let parquet_exec = ParquetExec::builder( + let parquet_exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file(partitioned_file), - ) - .build(); + Arc::new(ParquetConfig::default()), + ))); let mut results = parquet_exec.execute(0, state.task_ctx())?; let batch = results.next().await.unwrap(); @@ -1991,7 +1728,7 @@ mod tests { .await; // should have a pruning predicate - let pruning_predicate = &rt.parquet_exec.pruning_predicate; + let pruning_predicate = &rt.parquet_config.pruning_predicate; assert!(pruning_predicate.is_some()); // convert to explain plan form @@ -2032,7 +1769,7 @@ mod tests { .round_trip(vec![batches.clone()]) .await; - let pruning_predicate = &rt0.parquet_exec.pruning_predicate; + let pruning_predicate = &rt0.parquet_config.pruning_predicate; assert!(pruning_predicate.is_some()); let display0 = displayable(rt0.parquet_exec.as_ref()) @@ -2074,9 +1811,9 @@ mod tests { .await; // should have a pruning predicate - let pruning_predicate = &rt1.parquet_exec.pruning_predicate; + let pruning_predicate = &rt1.parquet_config.pruning_predicate; assert!(pruning_predicate.is_some()); - let pruning_predicate = &rt2.parquet_exec.pruning_predicate; + let pruning_predicate = &rt2.parquet_config.predicate; assert!(pruning_predicate.is_some()); // convert to explain plan form @@ -2117,14 +1854,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_config.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_config.predicate.as_ref(); let filter_phys = logical2physical(&filter, rt.parquet_exec.schema().as_ref()); assert_eq!(predicate.unwrap().to_string(), filter_phys.to_string()); } @@ -2152,11 +1889,11 @@ mod tests { .await; // Should have a pruning predicate - let pruning_predicate = &rt.parquet_exec.pruning_predicate; + let pruning_predicate = &rt.parquet_config.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 @@ -2466,7 +2203,12 @@ mod tests { let size_hint_calls = reader_factory.metadata_size_hint_calls.clone(); - let exec = ParquetExec::builder( + let source_config = Arc::new( + ParquetConfig::default() + .with_parquet_file_reader_factory(reader_factory) + .with_metadata_size_hint(456), + ); + let exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( FileScanConfig::new(store_url, schema) .with_file( PartitionedFile { @@ -2499,10 +2241,8 @@ mod tests { extensions: None, metadata_size_hint: None, }), - ) - .with_parquet_file_reader_factory(reader_factory) - .with_metadata_size_hint(456) - .build(); + source_config, + ))); let exec = Arc::new(exec); let res = collect(exec, ctx.task_ctx()).await.unwrap(); diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index b27cf9c5f833..a5bc66ac3318 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 ParquetConfig 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 ParquetConfig). `map_partial_batch` thus uses /// `table_schema` to create the resulting RecordBatch (as it could be operating /// on any fields in the schema). /// @@ -441,16 +441,18 @@ 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::data_source::FileSourceConfig; use crate::datasource::listing::PartitionedFile; + use crate::datasource::object_store::ObjectStoreUrl; + use crate::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use crate::datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; + use crate::physical_plan::collect; + use crate::prelude::SessionContext; + use datafusion_common::record_batch; + use datafusion_physical_plan::source::DataSourceExec; #[cfg(feature = "parquet")] use parquet::arrow::ArrowWriter; use tempfile::TempDir; @@ -500,14 +502,16 @@ mod tests { let f2 = Field::new("extra_column", DataType::Utf8, true); let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); + let base_conf = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema) + .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 source_config = Arc::new( + ParquetConfig::default() + .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})), + ); + + let source = Arc::new(FileSourceConfig::new(base_conf, source_config)); + let parquet_exec = DataSourceExec::new(source); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index 1ffe54e4b06c..2a4eee194405 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -503,11 +503,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/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index c44200a492eb..f6e9e95450a0 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -964,14 +964,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]", /// ``` /// /// 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]", /// ``` fn remove_dist_changing_operators( mut distribution_context: DistributionContext, @@ -996,7 +996,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=csv", /// ``` /// /// This function converts plan above to the following: @@ -1005,7 +1005,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=csv", /// ``` fn replace_order_preserving_variants( mut context: DistributionContext, @@ -1107,8 +1107,7 @@ fn get_repartition_requirement_status( Precision::Absent => true, }; let is_hash = matches!(requirement, Distribution::HashPartitioned(_)); - // Hash re-partitioning is necessary when the input has more than one - // partitions: + // Hash re-partitioning is necessary when the input has more than one partition: let multi_partitions = child.output_partitioning().partition_count() > 1; let roundrobin_sensible = roundrobin_beneficial && roundrobin_beneficial_stats; needs_alignment |= is_hash && (multi_partitions || roundrobin_sensible); @@ -1404,17 +1403,18 @@ struct JoinKeyPairs { /// Keeps track of parent required key orderings. type PlanWithKeyRequirements = PlanContext>>; -/// Since almost all of these tests explicitly use `ParquetExec` they only run with the parquet feature flag on +/// Since almost all of these tests explicitly use `ParquetConfig` they only run with the parquet feature flag on #[cfg(feature = "parquet")] #[cfg(test)] pub(crate) mod tests { use std::ops::Deref; use super::*; + use crate::datasource::data_source::FileSourceConfig; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; + use crate::datasource::physical_plan::{CsvConfig, FileScanConfig, ParquetConfig}; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::test_utils::{ check_integrity, coalesce_partitions_exec, repartition_exec, @@ -1426,7 +1426,6 @@ pub(crate) mod tests { use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; - use datafusion_physical_optimizer::output_requirements::OutputRequirements; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::ScalarValue; @@ -1436,6 +1435,8 @@ pub(crate) mod tests { expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, }; use datafusion_physical_expr_common::sort_expr::LexRequirement; + use datafusion_physical_optimizer::output_requirements::OutputRequirements; + use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::PlanProperties; /// Models operators like BoundedWindowExec that require an input @@ -1546,87 +1547,78 @@ pub(crate) mod tests { ])) } - fn parquet_exec() -> Arc { + fn parquet_exec() -> Arc { parquet_exec_with_sort(vec![]) } /// create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( output_ordering: Vec, - ) -> Arc { - ParquetExec::builder( + ) -> Arc { + let base_config = FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .build_arc() + .with_output_ordering(output_ordering); + let source_config = Arc::new(ParquetConfig::default()); + FileSourceConfig::new_exec(base_config, source_config) } - 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( + ) -> Arc { + let base_config = 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), - ) - .build_arc() + .with_output_ordering(output_ordering); + let source_config = Arc::new(ParquetConfig::default()); + FileSourceConfig::new_exec(base_config, source_config) } - fn csv_exec() -> Arc { - csv_exec_with_sort(vec![]) + fn data_source_exec_csv() -> Arc { + source_exec_with_csv_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 source_exec_with_csv_sort( + output_ordering: Vec, + ) -> Arc { + let conf = + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED); + let source_config = Arc::new(CsvConfig::new(false, b',', b'"')); + FileSourceConfig::new_exec(conf, source_config) } - fn csv_exec_multiple() -> Arc { - csv_exec_multiple_sorted(vec![]) + fn data_source_exec_csv_multiple() -> Arc { + data_source_exec_csv_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 data_source_exec_csv_multiple_sorted( + output_ordering: Vec, + ) -> Arc { + let conf = + 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_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED); + let source_config = Arc::new(CsvConfig::new(false, b',', b'"')); + + FileSourceConfig::new_exec(conf, source_config) } fn projection_exec_with_alias( @@ -2017,14 +2009,14 @@ pub(crate) mod tests { 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![ @@ -2033,14 +2025,14 @@ pub(crate) mod tests { 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); @@ -2081,14 +2073,14 @@ pub(crate) mod tests { 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 _ => @@ -2098,14 +2090,14 @@ pub(crate) mod tests { 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); @@ -2157,13 +2149,13 @@ pub(crate) mod tests { "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); @@ -2183,13 +2175,13 @@ pub(crate) mod tests { "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); @@ -2238,13 +2230,13 @@ pub(crate) mod tests { "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); @@ -2280,12 +2272,12 @@ pub(crate) mod tests { "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); @@ -2333,12 +2325,12 @@ pub(crate) mod tests { "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); @@ -2458,19 +2450,19 @@ pub(crate) mod tests { "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); @@ -2602,19 +2594,19 @@ pub(crate) mod tests { "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); @@ -2742,19 +2734,19 @@ pub(crate) mod tests { "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); @@ -2820,16 +2812,16 @@ pub(crate) mod tests { "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 @@ -2849,16 +2841,16 @@ pub(crate) mod tests { "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); @@ -2872,16 +2864,16 @@ pub(crate) mod tests { "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 @@ -2903,16 +2895,16 @@ pub(crate) mod tests { "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); @@ -2943,16 +2935,16 @@ pub(crate) mod tests { "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![ @@ -2963,16 +2955,16 @@ pub(crate) mod tests { "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!() @@ -2987,16 +2979,16 @@ pub(crate) mod tests { "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![ @@ -3009,16 +3001,16 @@ pub(crate) mod tests { "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!() @@ -3093,14 +3085,14 @@ pub(crate) mod tests { "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); @@ -3116,7 +3108,7 @@ pub(crate) mod tests { "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]", @@ -3126,7 +3118,7 @@ pub(crate) mod tests { "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); @@ -3157,7 +3149,7 @@ pub(crate) mod tests { 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); @@ -3169,7 +3161,7 @@ pub(crate) mod tests { "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); @@ -3205,12 +3197,12 @@ pub(crate) mod tests { "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); @@ -3248,12 +3240,12 @@ pub(crate) mod tests { "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; @@ -3288,7 +3280,7 @@ pub(crate) mod tests { "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); @@ -3307,7 +3299,7 @@ pub(crate) mod tests { "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); @@ -3327,7 +3319,7 @@ pub(crate) mod tests { "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); @@ -3350,7 +3342,7 @@ pub(crate) mod tests { "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); @@ -3377,7 +3369,7 @@ pub(crate) mod tests { // 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); @@ -3408,7 +3400,7 @@ pub(crate) mod tests { "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); @@ -3422,12 +3414,12 @@ pub(crate) mod tests { 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); @@ -3449,7 +3441,7 @@ pub(crate) mod tests { // 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); @@ -3474,7 +3466,7 @@ pub(crate) mod tests { // 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); @@ -3482,7 +3474,7 @@ pub(crate) mod tests { 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); @@ -3504,8 +3496,8 @@ pub(crate) mod tests { 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); @@ -3514,8 +3506,8 @@ pub(crate) mod tests { "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); @@ -3541,7 +3533,7 @@ pub(crate) mod tests { "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); @@ -3579,11 +3571,11 @@ pub(crate) mod tests { "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); @@ -3616,7 +3608,7 @@ pub(crate) mod tests { // 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); @@ -3627,7 +3619,7 @@ pub(crate) mod tests { // 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); @@ -3659,7 +3651,7 @@ pub(crate) mod tests { "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); @@ -3692,7 +3684,7 @@ pub(crate) mod tests { "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); @@ -3715,7 +3707,7 @@ pub(crate) mod tests { // 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); @@ -3726,7 +3718,7 @@ pub(crate) mod tests { "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); @@ -3762,7 +3754,7 @@ pub(crate) mod tests { "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); @@ -3773,7 +3765,7 @@ pub(crate) mod tests { "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); @@ -3784,19 +3776,19 @@ pub(crate) mod tests { fn parallelization_single_partition() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec(), alias); + let plan_csv = aggregate_exec_with_alias(data_source_exec_csv(), alias); let expected_parquet = [ "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); @@ -3821,7 +3813,8 @@ pub(crate) mod tests { 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!( @@ -3838,7 +3831,7 @@ pub(crate) mod tests { 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; @@ -3857,7 +3850,7 @@ pub(crate) mod tests { } #[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 = [ @@ -3873,14 +3866,14 @@ pub(crate) mod tests { "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 { @@ -3890,24 +3883,17 @@ pub(crate) mod tests { &expected_partitioned[..] }; - 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) + let conf = + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) .with_newlines_in_values(false) - .with_file_compression_type(compression_type) - .build(), - ), + .with_file_compression_type(compression_type); + let source_config = Arc::new(CsvConfig::new(false, b',', b'"')); + + let source = FileSourceConfig::new_exec(conf, source_config); + + let plan = aggregate_exec_with_alias( + source, vec![("a".to_string(), "a".to_string())], ); assert_optimized!(expected, plan, true, false, 2, true, 10, false); @@ -3920,21 +3906,21 @@ pub(crate) mod tests { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); + let plan_csv = aggregate_exec_with_alias(data_source_exec_csv_multiple(), alias); let expected_parquet = [ "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 - "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); @@ -3946,21 +3932,21 @@ pub(crate) mod tests { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); + let plan_csv = aggregate_exec_with_alias(data_source_exec_csv_multiple(), alias); let expected_parquet = [ "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 - "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); @@ -3976,7 +3962,7 @@ pub(crate) mod tests { options: SortOptions::default(), }]); let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); - let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); + let plan_csv = limit_exec(sort_exec(sort_key, data_source_exec_csv(), false)); let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -3984,7 +3970,7 @@ pub(crate) mod tests { // 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", @@ -3992,7 +3978,7 @@ pub(crate) mod tests { // 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); @@ -4012,7 +3998,11 @@ pub(crate) mod tests { parquet_exec(), false, ))); - let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec(), false))); + let plan_csv = limit_exec(filter_exec(sort_exec( + sort_key, + data_source_exec_csv(), + false, + ))); let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -4024,7 +4014,7 @@ pub(crate) mod tests { "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", @@ -4036,7 +4026,7 @@ pub(crate) mod tests { "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); @@ -4052,7 +4042,7 @@ pub(crate) mod tests { alias.clone(), ); let plan_csv = aggregate_exec_with_alias( - limit_exec(filter_exec(limit_exec(csv_exec()))), + limit_exec(filter_exec(limit_exec(data_source_exec_csv()))), alias, ); @@ -4070,7 +4060,7 @@ pub(crate) mod tests { "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=[]", @@ -4086,7 +4076,7 @@ pub(crate) mod tests { "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); @@ -4097,25 +4087,25 @@ pub(crate) mod tests { #[test] fn parallelization_union_inputs() -> Result<()> { let plan_parquet = union_exec(vec![parquet_exec(); 5]); - let plan_csv = union_exec(vec![csv_exec(); 5]); + let plan_csv = union_exec(vec![data_source_exec_csv(); 5]); 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); @@ -4137,15 +4127,15 @@ pub(crate) mod tests { ); let plan_csv = sort_preserving_merge_exec( sort_key.clone(), - csv_exec_with_sort(vec![sort_key]), + source_exec_with_csv_sort(vec![sort_key]), ); // 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); @@ -4163,7 +4153,8 @@ pub(crate) mod tests { // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let input_parquet = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); - let input_csv = union_exec(vec![csv_exec_with_sort(vec![sort_key.clone()]); 2]); + let input_csv = + union_exec(vec![source_exec_with_csv_sort(vec![sort_key.clone()]); 2]); let plan_parquet = sort_preserving_merge_exec(sort_key.clone(), input_parquet); let plan_csv = sort_preserving_merge_exec(sort_key, input_csv); @@ -4172,14 +4163,14 @@ pub(crate) mod tests { 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); @@ -4201,18 +4192,18 @@ pub(crate) mod tests { sort_key.clone(), ); let plan_csv = sort_required_exec_with_req( - csv_exec_with_sort(vec![sort_key.clone()]), + source_exec_with_csv_sort(vec![sort_key.clone()]), sort_key, ); // 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); @@ -4247,14 +4238,14 @@ pub(crate) mod tests { 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); @@ -4276,8 +4267,10 @@ pub(crate) mod tests { ("c".to_string(), "c2".to_string()), ]; - let proj_csv = - projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); + let proj_csv = projection_exec_with_alias( + source_exec_with_csv_sort(vec![sort_key]), + alias_pairs, + ); let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c2", &proj_csv.schema()).unwrap(), options: SortOptions::default(), @@ -4286,14 +4279,14 @@ pub(crate) mod tests { 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); @@ -4310,14 +4303,14 @@ pub(crate) mod tests { " 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); @@ -4341,7 +4334,7 @@ pub(crate) mod tests { "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); @@ -4364,7 +4357,7 @@ pub(crate) mod tests { "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); @@ -4388,7 +4381,7 @@ pub(crate) mod tests { "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); @@ -4398,7 +4391,7 @@ pub(crate) mod tests { "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); @@ -4421,7 +4414,7 @@ pub(crate) mod tests { // 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); @@ -4449,7 +4442,7 @@ pub(crate) mod tests { "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); @@ -4460,7 +4453,7 @@ pub(crate) mod tests { "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); @@ -4480,7 +4473,7 @@ pub(crate) mod tests { 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); @@ -4502,7 +4495,7 @@ pub(crate) mod tests { // 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); @@ -4512,7 +4505,7 @@ pub(crate) mod tests { // 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(); @@ -4540,7 +4533,7 @@ pub(crate) mod tests { // 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); @@ -4549,7 +4542,7 @@ pub(crate) mod tests { // 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(); @@ -4576,7 +4569,7 @@ pub(crate) mod tests { 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); @@ -4606,7 +4599,7 @@ pub(crate) mod tests { "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); @@ -4621,12 +4614,12 @@ pub(crate) mod tests { 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); @@ -4644,7 +4637,7 @@ pub(crate) mod tests { " 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()); @@ -4652,7 +4645,7 @@ pub(crate) mod tests { "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/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index dd8e9d900b7d..f90ca3004b21 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -668,7 +668,9 @@ mod tests { }; use crate::physical_plan::{displayable, get_plan_string, Partitioning}; use crate::prelude::{SessionConfig, SessionContext}; - use crate::test::{csv_exec_ordered, csv_exec_sorted, stream_exec_ordered}; + use crate::test::{ + data_source_exec_csv_ordered, data_source_exec_csv_sorted, stream_exec_ordered, + }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -803,11 +805,11 @@ mod tests { 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); @@ -861,14 +863,14 @@ mod tests { " 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(()) @@ -885,11 +887,11 @@ mod tests { 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); @@ -912,11 +914,11 @@ mod tests { " 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); @@ -950,13 +952,13 @@ mod tests { " 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); @@ -994,14 +996,14 @@ mod tests { " 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); @@ -1045,18 +1047,18 @@ mod tests { " 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(()) @@ -1079,12 +1081,12 @@ mod tests { 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(()) @@ -1112,11 +1114,11 @@ mod tests { 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); @@ -1146,12 +1148,12 @@ mod tests { 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); @@ -1179,12 +1181,12 @@ mod tests { "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); @@ -1207,13 +1209,13 @@ mod tests { "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); @@ -1239,11 +1241,11 @@ mod tests { "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); @@ -1272,22 +1274,22 @@ mod tests { 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(()) @@ -1306,11 +1308,11 @@ mod tests { 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); @@ -1334,11 +1336,11 @@ mod tests { "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); @@ -1362,9 +1364,9 @@ mod tests { 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(); @@ -1394,9 +1396,9 @@ mod tests { 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(); @@ -1427,16 +1429,16 @@ mod tests { // 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(()) @@ -1467,18 +1469,18 @@ mod tests { 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(()) @@ -1509,18 +1511,18 @@ mod tests { 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(()) @@ -1559,15 +1561,15 @@ mod tests { 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(()) @@ -1602,21 +1604,21 @@ mod tests { 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(()) @@ -1642,16 +1644,16 @@ mod tests { 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(()) @@ -1693,14 +1695,14 @@ mod tests { // 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(()) @@ -1742,15 +1744,15 @@ mod tests { " 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(()) @@ -1780,14 +1782,14 @@ mod tests { " 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(()) @@ -1827,19 +1829,19 @@ mod tests { 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(()) @@ -1885,8 +1887,8 @@ mod tests { ); 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 @@ -1896,9 +1898,9 @@ mod tests { 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", ] } _ => { @@ -1907,9 +1909,9 @@ mod tests { "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", ] } }; @@ -1963,17 +1965,17 @@ mod tests { ); 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", ] } _ => { @@ -1982,9 +1984,9 @@ mod tests { "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", ] } }; @@ -2019,16 +2021,16 @@ mod tests { 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) @@ -2041,16 +2043,16 @@ mod tests { 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(()) @@ -2080,13 +2082,13 @@ mod tests { " 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(()) @@ -2116,12 +2118,12 @@ mod tests { " 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(()) @@ -2151,7 +2153,7 @@ mod tests { "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, @@ -2208,12 +2210,12 @@ mod tests { " 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); @@ -2224,7 +2226,7 @@ mod tests { async fn test_with_lost_ordering_bounded() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); + let source = data_source_exec_csv_sorted(&schema, sort_exprs); let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( repartition_rr, @@ -2237,12 +2239,12 @@ mod tests { " 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(()) @@ -2259,7 +2261,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_ordered(&schema, sort_exprs) + data_source_exec_csv_ordered(&schema, sort_exprs) }; let repartition_rr = repartition_exec(source); let repartition_hash = Arc::new(RepartitionExec::try_new( @@ -2282,7 +2284,7 @@ mod tests { " 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) @@ -2299,14 +2301,14 @@ mod tests { " 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 { @@ -2342,7 +2344,7 @@ mod tests { async fn test_do_not_pushdown_through_spm() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs.clone()); + let source = data_source_exec_csv_sorted(&schema, sort_exprs.clone()); let repartition_rr = repartition_exec(source); let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); let physical_plan = sort_exec(vec![sort_expr("b", &schema)], spm); @@ -2350,11 +2352,11 @@ mod tests { 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(()) @@ -2364,7 +2366,7 @@ mod tests { async fn test_pushdown_through_spm() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs.clone()); + let source = data_source_exec_csv_sorted(&schema, sort_exprs.clone()); let repartition_rr = repartition_exec(source); let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); let physical_plan = sort_exec( @@ -2379,11 +2381,11 @@ mod tests { 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(()) @@ -2393,7 +2395,7 @@ mod tests { async fn test_window_multi_layer_requirement() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, vec![]); + let source = data_source_exec_csv_sorted(&schema, vec![]); let sort = sort_exec(sort_exprs.clone(), source); let repartition = repartition_exec(sort); let repartition = spr_repartition_exec(repartition); @@ -2407,7 +2409,7 @@ mod tests { " 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]", @@ -2415,7 +2417,7 @@ mod tests { " 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); @@ -2489,7 +2491,7 @@ mod tests { ); 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); @@ -2524,7 +2526,7 @@ mod tests { // 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)]); @@ -2540,7 +2542,7 @@ mod tests { "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; @@ -2554,7 +2556,7 @@ mod tests { // 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 = @@ -2571,13 +2573,13 @@ mod tests { "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/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index d2d35c3877c1..5d98734cdb5d 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -24,7 +24,7 @@ use std::collections::HashMap; use std::sync::Arc; use super::output_requirements::OutputRequirementExec; -use crate::datasource::physical_plan::CsvExec; +use crate::datasource::physical_plan::CsvConfig; use crate::error::Result; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::filter::FilterExec; @@ -33,7 +33,6 @@ use crate::physical_plan::joins::{ CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, SymmetricHashJoinExec, }; -use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; @@ -54,8 +53,11 @@ use datafusion_physical_expr::{ use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; +use crate::datasource::data_source::FileSourceConfig; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use itertools::Itertools; /// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to @@ -105,10 +107,47 @@ pub fn remove_unnecessary_projections( } // If it does, check if we can push it under its child(ren): let input = projection.input().as_any(); - if let Some(csv) = input.downcast_ref::() { - try_swapping_with_csv(projection, csv) - } else if let Some(memory) = input.downcast_ref::() { - try_swapping_with_memory(projection, memory)? + let is_csv = input + .downcast_ref::() + .and_then(|exec| { + exec.source() + .as_any() + .downcast_ref::() + .and_then(|file_config| { + file_config + .source_config() + .as_any() + .downcast_ref::() + .map(|_| true) + }) + }) + .unwrap_or(false); + + let is_memory = input + .downcast_ref::() + .and_then(|exec| { + exec.source() + .as_any() + .downcast_ref::() + .map(|_| true) + }) + .unwrap_or(false); + + if is_csv { + try_swapping_with_csv( + projection, + input.downcast_ref::().unwrap(), + ) + } else if is_memory { + let data_source = input.downcast_ref::().unwrap(); + let memory = data_source.source(); + try_swapping_with_memory( + projection, + memory + .as_any() + .downcast_ref::() + .unwrap(), + )? } else if let Some(child_projection) = input.downcast_ref::() { let maybe_unified = try_unifying_projections(projection, child_projection)?; return if let Some(new_plan) = maybe_unified { @@ -166,15 +205,17 @@ pub fn remove_unnecessary_projections( } /// Tries to embed `projection` to its input (`csv`). If possible, returns -/// [`CsvExec`] as the top plan. Otherwise, returns `None`. +/// [`DataSourceExec`] as the top plan. Otherwise, returns `None`. fn try_swapping_with_csv( projection: &ProjectionExec, - csv: &CsvExec, + csv: &DataSourceExec, ) -> Option> { // 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. + // This process can be moved into DataSourceExec, but it would be an overlap of their responsibility. all_alias_free_columns(projection.expr()).then(|| { - let mut file_scan = csv.base_config().clone(); + let source = csv.source(); + let csv_config = source.as_any().downcast_ref::().unwrap(); + let mut file_scan = csv_config.base_config().clone(); let new_projections = new_projections_for_columns( projection, &file_scan @@ -182,43 +223,32 @@ fn try_swapping_with_csv( .unwrap_or((0..csv.schema().fields().len()).collect()), ); file_scan.projection = Some(new_projections); - - Arc::new( - CsvExec::builder(file_scan) - .with_has_header(csv.has_header()) - .with_delimeter(csv.delimiter()) - .with_quote(csv.quote()) - .with_escape(csv.escape()) - .with_comment(csv.comment()) - .with_newlines_in_values(csv.newlines_in_values()) - .with_file_compression_type(csv.file_compression_type) - .build(), - ) as _ + FileSourceConfig::new_exec(file_scan, Arc::clone(csv_config.source_config())) as _ }) } /// Tries to embed `projection` to its input (`memory`). If possible, returns -/// [`MemoryExec`] as the top plan. Otherwise, returns `None`. +/// [`DataSourceExec`] as the top plan. Otherwise, returns `None`. fn try_swapping_with_memory( projection: &ProjectionExec, - memory: &MemoryExec, + memory: &MemorySourceConfig, ) -> Result>> { // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into MemoryExec, but it would be an overlap of their responsibility. + // This process can be moved into DataSourceExec, but it would be an overlap of their responsibility. all_alias_free_columns(projection.expr()) .then(|| { - let all_projections = (0..memory.schema().fields().len()).collect(); + let all_projections = (0..memory.original_schema().fields().len()).collect(); let new_projections = new_projections_for_columns( projection, memory.projection().as_ref().unwrap_or(&all_projections), ); - MemoryExec::try_new( + let source = Arc::new(MemorySourceConfig::try_new( memory.partitions(), memory.original_schema(), Some(new_projections), - ) - .map(|e| Arc::new(e) as _) + )?); + Ok(Arc::new(DataSourceExec::new(source)) as _) }) .transpose() } @@ -292,7 +322,7 @@ fn try_unifying_projections( .unwrap(); }); - // Merging these projections is not beneficial, e.g + // Merging these projections is not beneficial, e.g. // If an expression is not trivial and it is referred more than 1, unifies projections will be // beneficial as caching mechanism for non-trivial computations. // See discussion in: https://github.com/apache/datafusion/issues/8296 @@ -1720,21 +1750,15 @@ mod tests { 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(), - ) + let conf = + 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_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED); + + let source_config = Arc::new(CsvConfig::new(false, 0, 0)); + FileSourceConfig::new_exec(conf, source_config) } fn create_projecting_csv_exec() -> Arc { @@ -1744,21 +1768,15 @@ mod tests { 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(), - ) + let conf = + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![3, 2, 1])) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED); + + let source_config = Arc::new(CsvConfig::new(false, 0, 0)); + FileSourceConfig::new_exec(conf, source_config) } fn create_projecting_memory_exec() -> Arc { @@ -1770,7 +1788,10 @@ mod tests { Field::new("e", DataType::Int32, true), ])); - Arc::new(MemoryExec::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap()) + let source = Arc::new( + MemorySourceConfig::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap(), + ); + Arc::new(DataSourceExec::new(source)) } #[test] @@ -1786,7 +1807,7 @@ mod tests { 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); @@ -1794,7 +1815,7 @@ mod tests { 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); @@ -1815,20 +1836,24 @@ mod tests { 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() @@ -1986,7 +2011,7 @@ mod tests { 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); @@ -1995,7 +2020,7 @@ mod tests { 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); @@ -2039,7 +2064,7 @@ mod tests { 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); @@ -2049,7 +2074,7 @@ mod tests { 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); @@ -2116,7 +2141,7 @@ mod tests { 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); @@ -2126,7 +2151,7 @@ mod tests { 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); @@ -2164,7 +2189,7 @@ mod tests { 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); @@ -2174,7 +2199,7 @@ mod tests { 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); @@ -2249,8 +2274,8 @@ mod tests { 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); @@ -2260,9 +2285,9 @@ mod tests { 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); @@ -2368,8 +2393,8 @@ mod tests { 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); @@ -2379,8 +2404,8 @@ mod tests { 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(()) @@ -2468,7 +2493,7 @@ mod tests { )?); 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); @@ -2476,7 +2501,7 @@ mod tests { 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( @@ -2493,7 +2518,7 @@ mod tests { 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(()) @@ -2525,7 +2550,7 @@ mod tests { 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); @@ -2535,7 +2560,7 @@ mod tests { 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); @@ -2592,7 +2617,7 @@ mod tests { 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); @@ -2602,7 +2627,7 @@ mod tests { 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); @@ -2642,7 +2667,7 @@ mod tests { 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); @@ -2652,7 +2677,7 @@ mod tests { 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); @@ -2677,9 +2702,9 @@ mod tests { 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); @@ -2689,11 +2714,11 @@ mod tests { 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/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 96b2454fa330..04a94c10540f 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -285,9 +285,10 @@ pub(crate) fn replace_with_order_preserving_variants( mod tests { use super::*; + use crate::datasource::data_source::FileSourceConfig; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; - use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; + use crate::datasource::physical_plan::{CsvConfig, FileScanConfig}; use crate::physical_optimizer::test_utils::check_integrity; use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; use crate::physical_plan::filter::FilterExec; @@ -426,7 +427,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); @@ -447,7 +448,7 @@ mod tests { " 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -464,13 +465,13 @@ mod tests { " 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -494,7 +495,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); @@ -538,7 +539,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -564,7 +565,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC]", @@ -574,7 +575,7 @@ mod tests { " 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -598,7 +599,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let filter = filter_exec(repartition_rr); @@ -623,7 +624,7 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -642,14 +643,14 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -673,7 +674,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); @@ -701,7 +702,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -722,7 +723,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -730,7 +731,7 @@ mod tests { " 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -754,7 +755,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); @@ -786,7 +787,7 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -809,7 +810,7 @@ mod tests { " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -818,7 +819,7 @@ mod tests { " 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -842,7 +843,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); @@ -867,7 +868,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -887,7 +888,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -913,7 +914,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); @@ -944,7 +945,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -967,7 +968,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", @@ -976,7 +977,7 @@ mod tests { " 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -1000,7 +1001,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); @@ -1028,7 +1029,7 @@ mod tests { " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -1046,7 +1047,7 @@ mod tests { " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -1072,7 +1073,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); @@ -1093,7 +1094,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -1110,13 +1111,13 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -1140,7 +1141,7 @@ mod tests { let source = if source_unbounded { stream_exec_ordered(&schema, sort_exprs) } else { - csv_exec_sorted(&schema, sort_exprs) + data_source_exec_csv_sorted(&schema, sort_exprs) }; let repartition_rr = repartition_exec_round_robin(source); let repartition_hash = repartition_exec_hash(repartition_rr); @@ -1184,7 +1185,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -1211,7 +1212,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [c@1 ASC]", @@ -1222,7 +1223,7 @@ mod tests { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; assert_optimized_in_all_boundedness_situations!( expected_input_unbounded, @@ -1247,7 +1248,7 @@ mod tests { let left_source = if source_unbounded { stream_exec_ordered(&schema, left_sort_exprs) } else { - csv_exec_sorted(&schema, left_sort_exprs) + data_source_exec_csv_sorted(&schema, left_sort_exprs) }; let left_repartition_rr = repartition_exec_round_robin(left_source); let left_repartition_hash = repartition_exec_hash(left_repartition_rr); @@ -1258,7 +1259,7 @@ mod tests { let right_source = if source_unbounded { stream_exec_ordered(&schema, right_sort_exprs) } else { - csv_exec_sorted(&schema, right_sort_exprs) + data_source_exec_csv_sorted(&schema, right_sort_exprs) }; let right_repartition_rr = repartition_exec_round_robin(right_source); let right_repartition_hash = repartition_exec_hash(right_repartition_rr); @@ -1299,11 +1300,11 @@ mod tests { " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -1330,11 +1331,11 @@ mod tests { " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -1494,31 +1495,24 @@ mod tests { // creates a csv exec source for the test purposes // projection and has_header parameters are given static due to testing needs - fn csv_exec_sorted( + fn data_source_exec_csv_sorted( schema: &SchemaRef, sort_exprs: impl IntoIterator, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); let projection: Vec = vec![0, 2, 3]; - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("file_path".to_string(), 100)) - .with_projection(Some(projection)) - .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(), + let conf = FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), ) + .with_file(PartitionedFile::new("file_path".to_string(), 100)) + .with_projection(Some(projection)) + .with_output_ordering(vec![sort_exprs]) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED); + + let source_config = Arc::new(CsvConfig::new(true, 0, b'"')); + FileSourceConfig::new_exec(conf, source_config) } } diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index f4b0f7c6069b..eb131f76d327 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -457,7 +457,7 @@ mod tests { 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(()) @@ -479,7 +479,7 @@ mod tests { 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); @@ -498,7 +498,7 @@ mod tests { 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); @@ -518,7 +518,7 @@ mod tests { 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. @@ -537,7 +537,7 @@ mod tests { limit.as_ref(), vec![ "LocalLimitExec: fetch=100", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ], ); assert_sanity_check(&limit, true); @@ -578,10 +578,10 @@ mod tests { "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); @@ -625,9 +625,9 @@ mod tests { "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. @@ -671,10 +671,10 @@ mod tests { "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/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 9156301393c0..305fa9481088 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -24,7 +24,7 @@ use std::fmt::Formatter; use std::sync::Arc; use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use crate::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use crate::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; use crate::error::Result; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; @@ -34,7 +34,6 @@ use crate::physical_plan::filter::FilterExec; use crate::physical_plan::joins::utils::{JoinFilter, JoinOn}; use crate::physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -56,9 +55,12 @@ use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, }; +use crate::datasource::data_source::FileSourceConfig; use async_trait::async_trait; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; async fn register_current_csv( ctx: &SessionContext, @@ -216,7 +218,9 @@ pub fn coalesce_partitions_exec(input: Arc) -> Arc Arc { - Arc::new(MemoryExec::try_new(&[vec![]], schema.clone(), None).unwrap()) + let source = + Arc::new(MemorySourceConfig::try_new(&[vec![]], schema.clone(), None).unwrap()); + Arc::new(DataSourceExec::new(source)) } pub fn hash_join_exec( @@ -283,12 +287,13 @@ pub fn sort_preserving_merge_exec( } /// Create a non sorted parquet exec -pub fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( +pub fn parquet_exec(schema: &SchemaRef) -> Arc { + let base_config = FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() + .with_file(PartitionedFile::new("x".to_string(), 100)); + let source_config = Arc::new(ParquetConfig::default()); + + FileSourceConfig::new_exec(base_config, source_config) } // Created a sorted parquet exec @@ -298,12 +303,13 @@ pub fn parquet_exec_sorted( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - ParquetExec::builder( + let base_config = FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .build_arc() + .with_output_ordering(vec![sort_exprs]); + let source_config = Arc::new(ParquetConfig::default()); + + FileSourceConfig::new_exec(base_config, source_config) } pub fn union_exec(input: Vec>) -> Arc { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 47b31d2f4e2d..0e326f32360a 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,12 +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::memory::MemorySourceConfig; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; +use datafusion_physical_plan::source::DataSourceExec; 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}; @@ -1940,7 +1941,11 @@ 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)?; + let mem_exec = DataSourceExec::new(Arc::new(MemorySourceConfig::try_new( + &partitions, + schema, + projection, + )?)); Ok(Arc::new(mem_exec)) } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index b4167900d4c2..b508bce7f2f3 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -26,12 +26,13 @@ use std::io::{BufReader, BufWriter}; use std::path::Path; use std::sync::Arc; +use crate::datasource::data_source::FileSourceConfig; 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::{CsvConfig, FileScanConfig}; use crate::datasource::{MemTable, TableProvider}; use crate::error::Result; use crate::logical_expr::LogicalPlan; @@ -46,6 +47,7 @@ use datafusion_common::{DataFusionError, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalSortExpr}; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; @@ -79,8 +81,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()); @@ -92,18 +97,10 @@ pub fn scan_partitioned_csv(partitions: usize, work_dir: &Path) -> Result RecordBatch { } /// Created a sorted Csv exec -pub fn csv_exec_sorted( +pub fn data_source_exec_csv_sorted( schema: &SchemaRef, sort_exprs: impl IntoIterator, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) + let config = + 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(), - ) + .with_output_ordering(vec![sort_exprs]) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .with_newlines_in_values(false); + let source_config = Arc::new(CsvConfig::new(false, 0, 0)); + FileSourceConfig::new_exec(config, source_config) } // construct a stream partition for test purposes @@ -358,30 +344,18 @@ pub fn stream_exec_ordered( } /// Create a csv exec for tests -pub fn csv_exec_ordered( +pub fn data_source_exec_csv_ordered( schema: &SchemaRef, sort_exprs: impl IntoIterator, ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) + let config = + 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(), - ) + .with_output_ordering(vec![sort_exprs]); + let source_config = Arc::new(CsvConfig::new(true, 0, b'"')); + FileSourceConfig::new_exec(config, source_config) } /// A mock execution plan that simply returns the provided statistics diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 685ed14777b4..5501bb3cb892 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, ParquetConfig}; use crate::error::Result; use crate::logical_expr::execution_props::ExecutionProps; use crate::logical_expr::simplify::SimplifyContext; @@ -37,7 +37,8 @@ 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 crate::datasource::data_source::FileSourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; @@ -137,18 +138,18 @@ 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, @@ -177,28 +178,42 @@ impl TestParquetFile { 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_config = Arc::new(ParquetConfig::new( + Arc::clone(&scan_config.file_schema), + Some(Arc::clone(&physical_filter_expr)), + None, + parquet_options, + )); + let parquet_exec = FileSourceConfig::new_exec(scan_config, source_config); 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(), - ) + let source_config = + Arc::new(ParquetConfig::new_with_options(parquet_options)); + Ok(FileSourceConfig::new_exec(scan_config, source_config)) } } /// 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 + .source_config() + .as_any() + .downcast_ref::() + .is_some() + { + return maybe_file.metrics(); + } + } } for child in plan.children() { diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 09d0c8d5ca2e..b32794f7a4b5 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; // ======================================================================== @@ -301,16 +303,19 @@ 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 source = Arc::new( + MemorySourceConfig::try_new(&[vec![concat_input_record]], schema.clone(), None) + .unwrap(), ); + let usual_source = Arc::new(DataSourceExec::new(source)); - let running_source = Arc::new( - MemoryExec::try_new(&[input1.clone()], schema.clone(), None) + 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 b331388f4f3f..2a34a9d4ea32 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -14,22 +14,16 @@ // KIND, either express or implied. See the License for the // 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 +33,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,14 +424,19 @@ 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()); - let right = - Arc::new(MemoryExec::try_new(&[self.input2.clone()], schema2, None).unwrap()); - (left, right) + let left = Arc::new( + MemorySourceConfig::try_new(&[self.input1.clone()], schema1, None).unwrap(), + ); + let right = Arc::new( + MemorySourceConfig::try_new(&[self.input2.clone()], schema2, None).unwrap(), + ); + ( + Arc::new(DataSourceExec::new(left)), + Arc::new(DataSourceExec::new(right)), + ) } fn join_filter(&self) -> Option { diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 4e895920dd3d..f828b51178c3 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, sorts::sort_preserving_merge::SortPreservingMergeExec, }; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; #[tokio::test] @@ -116,7 +117,8 @@ async fn run_merge_test(input: Vec>) { }, }]); - let exec = MemoryExec::try_new(&input, schema, None).unwrap(); + let config = MemorySourceConfig::try_new(&input, schema, None).unwrap(); + let exec = DataSourceExec::new(Arc::new(config)); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); let session_config = SessionConfig::new().with_batch_size(batch_size); diff --git a/datafusion/core/tests/fuzz_cases/pruning.rs b/datafusion/core/tests/fuzz_cases/pruning.rs index 3725e6d908e6..d9617e2b654d 100644 --- a/datafusion/core/tests/fuzz_cases/pruning.rs +++ b/datafusion/core/tests/fuzz_cases/pruning.rs @@ -20,16 +20,17 @@ use std::sync::{Arc, OnceLock}; use arrow_array::{Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; use bytes::{BufMut, Bytes, BytesMut}; +use datafusion::datasource::data_source::FileSourceConfig; +use datafusion::datasource::physical_plan::ParquetConfig; use datafusion::{ - datasource::{ - listing::PartitionedFile, - physical_plan::{parquet::ParquetExecBuilder, FileScanConfig}, - }, + datasource::{listing::PartitionedFile, physical_plan::FileScanConfig}, prelude::*, }; +use datafusion_common::config::TableParquetOptions; use datafusion_common::DFSchema; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{collect, filter::FilterExec, ExecutionPlan}; use itertools::Itertools; use object_store::{memory::InMemory, path::Path, ObjectStore, PutPayload}; @@ -316,11 +317,23 @@ async fn execute_with_predicate( }) .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_conf = if prune_stats { + ParquetConfig::new( + Arc::clone(&scan.file_schema), + Some(predicate.clone()), + None, + TableParquetOptions::default(), + ) + } else { + ParquetConfig::new( + Arc::clone(&scan.file_schema), + None, + None, + TableParquetOptions::default(), + ) + }; + let source = Arc::new(FileSourceConfig::new(scan, Arc::new(parquet_conf))); + let exec = Arc::new(DataSourceExec::new(source)) as Arc; 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..6403eb0f71ed 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -24,13 +24,14 @@ 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 datafusion_physical_plan::source::DataSourceExec; use rand::Rng; use std::sync::Arc; use test_utils::{batches_to_vec, partitions_to_sorted_vec}; @@ -123,7 +124,8 @@ impl SortTest { }, }]); - let exec = MemoryExec::try_new(&input, schema, None).unwrap(); + let config = MemorySourceConfig::try_new(&input, schema, None).unwrap(); + let exec = DataSourceExec::new(Arc::new(config)); let sort = Arc::new(SortExec::new(sort, Arc::new(exec))); let session_config = SessionConfig::new(); 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 67666f5d7a1c..fdd9a62e7ab8 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -22,7 +22,6 @@ 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::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ create_window_expr, schema_add_window_field, BoundedWindowAggExec, WindowAggExec, @@ -51,6 +50,8 @@ use datafusion_functions_window::nth_value::{ }; use datafusion_functions_window::rank::{dense_rank_udwf, rank_udwf}; 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}; @@ -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,12 @@ 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( + let memory_conf = Arc::new(MemorySourceConfig::try_new( &[batches.clone()], schema.clone(), None, )?); + let memory_exec = Arc::new(DataSourceExec::new(memory_conf)); // Different window functions to test causality let window_functions = vec![ @@ -638,10 +640,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 +666,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 c7514d1c24b1..d57c05d1868e 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -24,7 +24,6 @@ use arrow_schema::SortOptions; use async_trait::async_trait; use datafusion::assert_batches_eq; use datafusion::physical_optimizer::PhysicalOptimizerRule; -use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::streaming::PartitionStream; use datafusion_execution::memory_pool::{ GreedyMemoryPool, MemoryPool, TrackConsumersPool, @@ -51,6 +50,8 @@ use datafusion_common::{assert_contains, Result}; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_catalog::Session; use datafusion_execution::TaskContext; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use test_utils::AccessLogGenerator; #[cfg(test)] @@ -239,15 +240,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], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+----------------------------------------------------------------------------------------------------------------+", ] ) .run() @@ -286,15 +287,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 |", + "| | |", + "+---------------+-------------------------------------------------------------------------------------------------------------+", ] ); @@ -842,10 +843,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..1cb273ea5af2 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -24,16 +24,18 @@ use arrow::array::{ArrayRef, Int64Array, Int8Array, StringArray}; use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow::record_batch::RecordBatch; use datafusion::assert_batches_sorted_eq; +use datafusion::datasource::data_source::FileSourceConfig; 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, ParquetConfig, ParquetFileMetrics, ParquetFileReaderFactory, }; use datafusion::physical_plan::collect; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::prelude::SessionContext; use datafusion_common::Result; +use datafusion_physical_plan::source::DataSourceExec; use bytes::Bytes; use futures::future::BoxFuture; @@ -75,19 +77,22 @@ 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 base_config = FileScanConfig::new( + // just any url that doesn't point to in memory object store + ObjectStoreUrl::local_filesystem(), + file_schema, ) - .build() - .with_parquet_file_reader_factory(Arc::new(InMemoryParquetFileReaderFactory( - Arc::clone(&in_memory_object_store), - ))); + .with_file_group(file_group); + let source_config = Arc::new( + ParquetConfig::default() + // prepare the scan + .with_parquet_file_reader_factory(Arc::new( + InMemoryParquetFileReaderFactory(Arc::clone(&in_memory_object_store)), + )), + ); + + let parquet_exec = + DataSourceExec::new(Arc::new(FileSourceConfig::new(base_config, source_config))); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); diff --git a/datafusion/core/tests/parquet/external_access_plan.rs b/datafusion/core/tests/parquet/external_access_plan.rs index 61a9e9b5757c..7f7ff13ea80c 100644 --- a/datafusion/core/tests/parquet/external_access_plan.rs +++ b/datafusion/core/tests/parquet/external_access_plan.rs @@ -15,26 +15,32 @@ // 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::data_source::FileSourceConfig; 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, ParquetConfig}; use datafusion::prelude::SessionContext; +use datafusion_common::config::TableParquetOptions; 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 +280,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 +340,29 @@ 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_config = 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(ParquetConfig::new( + Arc::clone(&config.file_schema), + Some(predicate), + None, + TableParquetOptions::default(), + )) + } else { + Arc::new(ParquetConfig::default()) + }; - let plan: Arc = builder.build_arc(); + let plan: Arc = + FileSourceConfig::new_exec(config, source_config); - // 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..d470040eefcb 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -18,13 +18,14 @@ use std::fs; use std::sync::Arc; +use datafusion::datasource::data_source::FileSourceConfig; 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 +34,9 @@ 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 tempfile::tempdir; #[tokio::test] @@ -149,7 +150,9 @@ 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; @@ -160,7 +163,9 @@ 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; @@ -171,7 +176,9 @@ 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; diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 65bfd0340125..85b7c1fc4054 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -17,12 +17,13 @@ use crate::parquet::Unit::Page; use crate::parquet::{ContextWithParquet, Scenario}; +use std::sync::Arc; 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, ParquetConfig}; use datafusion::execution::context::SessionState; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; @@ -32,11 +33,14 @@ use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; +use datafusion::datasource::data_source::FileSourceConfig; +use datafusion_common::config::TableParquetOptions; +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 +75,19 @@ 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 base_config = + FileScanConfig::new(object_store_url, schema).with_file(partitioned_file); + let source_config = Arc::new( + ParquetConfig::new( + Arc::clone(&base_config.file_schema), + Some(predicate), + None, + TableParquetOptions::default(), + ) + .with_enable_page_index(true), + ); + + DataSourceExec::new(Arc::new(FileSourceConfig::new(base_config, source_config))) } #[tokio::test] diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index af9411f40ecb..a47ad800fa5c 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -23,11 +23,13 @@ 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::data_source::FileSourceConfig; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use datafusion::physical_plan::collect; use datafusion::prelude::SessionContext; use datafusion_common::Result; use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_physical_plan::source::DataSourceExec; use object_store::path::Path; use object_store::ObjectMeta; @@ -59,11 +61,13 @@ async fn multi_parquet_coercion() { Field::new("c2", DataType::Int32, true), Field::new("c3", DataType::Float64, true), ])); - let parquet_exec = ParquetExec::builder( + let source_config = Arc::new(ParquetConfig::default()); + + let parquet_exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file_group(file_group), - ) - .build(); + source_config, + ))); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); @@ -113,12 +117,12 @@ async fn multi_parquet_coercion_projection() { Field::new("c2", DataType::Int32, true), Field::new("c3", DataType::Float64, true), ])); - let parquet_exec = ParquetExec::builder( + let parquet_exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file_group(file_group) .with_projection(Some(vec![1, 0, 2])), - ) - .build(); + Arc::new(ParquetConfig::default()), + ))); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); diff --git a/datafusion/core/tests/parquet/utils.rs b/datafusion/core/tests/parquet/utils.rs index d8d2b2fbb8a5..cca2b7b3ae4c 100644 --- a/datafusion/core/tests/parquet/utils.rs +++ b/datafusion/core/tests/parquet/utils.rs @@ -17,11 +17,13 @@ //! Utilities for parquet tests -use datafusion::datasource::physical_plan::ParquetExec; +use datafusion::datasource::data_source::FileSourceConfig; +use datafusion::datasource::physical_plan::ParquetConfig; 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 +48,19 @@ 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 + .source_config() + .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..a6f54211a411 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 `ParquetConfig` and related components /// Run all tests that are found in the `parquet` directory mod parquet; 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 85076abdaf29..22bf65fb1436 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -18,8 +18,9 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use datafusion::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; use datafusion_common::config::ConfigOptions; use datafusion_execution::object_store::ObjectStoreUrl; @@ -35,6 +36,7 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::displayable; use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::ExecutionPlan; /// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected @@ -73,12 +75,13 @@ fn schema() -> SchemaRef { ])) } -fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( +fn parquet_exec(schema: &SchemaRef) -> Arc { + let base_config = FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() + .with_file(PartitionedFile::new("x".to_string(), 100)); + let source_config = Arc::new(ParquetConfig::default()); + + FileSourceConfig::new_exec(base_config, source_config) } fn partial_aggregate_exec( @@ -159,7 +162,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); @@ -179,7 +182,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); @@ -204,7 +207,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); @@ -240,7 +243,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); @@ -284,7 +287,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/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 6910db6285a3..00570468e023 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -41,15 +41,16 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::{ limited_distinct_aggregation::LimitedDistinctAggregation, PhysicalOptimizerRule, }; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, collect, displayable, expressions, limit::{GlobalLimitExec, LocalLimitExec}, - memory::MemoryExec, ExecutionPlan, }; -fn mock_data() -> Result> { +fn mock_data() -> Result> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), @@ -77,11 +78,11 @@ fn mock_data() -> Result> { ], )?; - Ok(Arc::new(MemoryExec::try_new( + Ok(MemorySourceConfig::try_new_exec( &[vec![batch]], Arc::clone(&schema), None, - )?)) + )?) } fn assert_plan_matches_expected( @@ -132,7 +133,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()]), @@ -158,7 +159,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)?; @@ -182,7 +183,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()]), @@ -199,7 +200,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)?; @@ -223,7 +224,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()]), @@ -241,7 +242,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)?; @@ -264,7 +265,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()]), @@ -290,7 +291,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)?; @@ -314,7 +315,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![]), @@ -331,7 +332,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)?; @@ -344,7 +345,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()]), @@ -361,7 +362,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)?; @@ -373,7 +374,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)?, @@ -399,7 +400,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)?; @@ -415,7 +416,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, @@ -433,7 +434,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)?; diff --git a/datafusion/core/tests/physical_optimizer/test_util.rs b/datafusion/core/tests/physical_optimizer/test_util.rs index 12cd08fb3db3..07209e6a419d 100644 --- a/datafusion/core/tests/physical_optimizer/test_util.rs +++ b/datafusion/core/tests/physical_optimizer/test_util.rs @@ -20,23 +20,23 @@ use std::sync::Arc; use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use datafusion::datasource::{ - listing::PartitionedFile, - physical_plan::{FileScanConfig, ParquetExec}, -}; +use datafusion::datasource::data_source::FileSourceConfig; +use datafusion::datasource::physical_plan::ParquetConfig; +use datafusion::datasource::{listing::PartitionedFile, physical_plan::FileScanConfig}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::source::DataSourceExec; /// create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( output_ordering: Vec, -) -> Arc { - ParquetExec::builder( +) -> Arc { + let base_config = FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .build_arc() + .with_output_ordering(output_ordering); + let source_config = Arc::new(ParquetConfig::default()); + FileSourceConfig::new_exec(base_config, source_config) } pub(crate) fn schema() -> SchemaRef { diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 5fb0b9852641..672abfa6b92a 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -617,7 +617,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(); @@ -657,12 +657,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 441af1639d9b..97c6a8ff0b34 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::ParquetConfig; use datafusion::{ assert_batches_sorted_eq, datasource::{ @@ -45,8 +45,10 @@ use datafusion_execution::config::SessionConfig; use async_trait::async_trait; use bytes::Bytes; use chrono::{TimeZone, Utc}; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion_expr::{col, lit, Expr, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; +use datafusion_physical_plan::source::DataSourceExec; use futures::stream::{self, BoxStream}; use object_store::{ path::Path, GetOptions, GetResult, GetResultPayload, ListResult, ObjectMeta, @@ -84,8 +86,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 + .source_config() + .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 45d467f133bf..d9b8a52f2725 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/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index a00bc4b1d571..b0235d920e9c 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -170,7 +170,8 @@ mod tests { 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::source::DataSourceExec; use arrow::array::Int32Array; use arrow::datatypes::{DataType, Field, Schema}; @@ -233,8 +234,8 @@ mod tests { } } - /// Mock data using a MemoryExec which has an exact count statistic - fn mock_data() -> Result> { + /// Mock data using a DataSourceExec 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), @@ -248,11 +249,11 @@ mod tests { ], )?; - Ok(Arc::new(MemoryExec::try_new( + Ok(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/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 c04211d679ca..cc4d7a04378c 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1314,9 +1314,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::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::RecordBatchStream; @@ -1340,13 +1341,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) @@ -2134,14 +2134,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. @@ -2176,7 +2176,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], @@ -2185,7 +2185,7 @@ mod tests { ], Arc::clone(&schema), None, - )?); + )?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups.clone(), @@ -2411,11 +2411,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, @@ -2534,11 +2534,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, @@ -2603,11 +2603,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, @@ -2693,11 +2690,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, 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 5f0b229ce92a..b854cf50b200 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -71,7 +71,7 @@ use crate::stream::RecordBatchStreamAdapter; /// [`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 @@ -80,7 +80,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 @@ -557,10 +557,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 @@ -729,7 +729,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 4e0b0bf820f2..70b082438e03 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1624,9 +1624,10 @@ impl Stream for HashJoinStream { #[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}; @@ -1642,7 +1643,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::*; @@ -1667,7 +1667,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( @@ -2069,9 +2069,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]), @@ -2141,9 +2141,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]), @@ -2195,9 +2195,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 _, @@ -2275,9 +2275,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)] @@ -2382,7 +2381,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()); @@ -2419,7 +2419,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()); @@ -3704,15 +3705,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 _, @@ -4002,27 +4002,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 _, @@ -4102,7 +4098,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 c69fa2888806..fa3a77722231 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -907,9 +907,10 @@ impl RecordBatchStream for NestedLoopJoinStr #[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}; @@ -947,8 +948,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 { @@ -962,10 +963,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 bcacc7dcae0f..50a931f743db 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -2436,7 +2436,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}; @@ -2447,12 +2447,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( @@ -2477,7 +2477,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( @@ -2502,7 +2502,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 @@ -2525,7 +2525,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( @@ -2534,7 +2534,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 37d6c0aff850..df46cb7c96ed 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::util::pretty::pretty_format_batches; @@ -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 521008ce9b02..0b9e54e3d875 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -28,6 +28,8 @@ use super::{ RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::execution_plan::{Boundedness, EmissionType}; +use crate::metrics::ExecutionPlanMetricsSet; +use crate::source::{DataSource, DataSourceExec}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -41,9 +43,8 @@ use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use futures::Stream; -/// Execution plan for reading in-memory batches of data -#[derive(Clone)] -pub struct MemoryExec { +/// Data source configuration for reading in-memory batches of data +pub struct MemorySourceConfig { /// The partitions to query partitions: Vec>, /// Schema representing the data before projection @@ -52,25 +53,31 @@ pub struct MemoryExec { projected_schema: SchemaRef, /// Optional projection projection: Option>, - // Sort information: one or more equivalent orderings - sort_information: Vec, + /// Plan Properties cache: PlanProperties, + /// Sort information: one or more equivalent orderings + sort_information: Vec, /// if partition sizes should be displayed show_sizes: bool, } -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() +impl DataSource for MemorySourceConfig { + fn open( + &self, + partition: usize, + _context: Arc, + ) -> Result { + Ok(Box::pin(MemoryStream::try_new( + self.partitions[partition].clone(), + Arc::clone(&self.projected_schema), + self.projection.clone(), + )?)) + } + + fn as_any(&self) -> &dyn Any { + self } -} -impl DisplayAs for MemoryExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { @@ -88,61 +95,16 @@ impl DisplayAs for MemoryExec { if self.show_sizes { write!( f, - "MemoryExec: partitions={}, partition_sizes={partition_sizes:?}{output_ordering}", + "partitions={}, partition_sizes={partition_sizes:?}{output_ordering}", partition_sizes.len(), ) } else { - write!(f, "MemoryExec: partitions={}", partition_sizes.len(),) + write!(f, "partitions={}", partition_sizes.len(),) } } } } -} -impl ExecutionPlan for MemoryExec { - fn name(&self) -> &'static str { - "MemoryExec" - } - - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - &self.cache - } - - fn children(&self) -> Vec<&Arc> { - // This is a leaf node and has no children - vec![] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> Result> { - // MemoryExec has no children - if children.is_empty() { - Ok(self) - } else { - internal_err!("Children cannot be replaced in {self:?}") - } - } - - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> Result { - Ok(Box::pin(MemoryStream::try_new( - self.partitions[partition].clone(), - Arc::clone(&self.projected_schema), - self.projection.clone(), - )?)) - } - - /// 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, @@ -150,10 +112,18 @@ impl ExecutionPlan for MemoryExec { self.projection.clone(), )) } + + fn metrics(&self) -> ExecutionPlanMetricsSet { + ExecutionPlanMetricsSet::new() + } + + fn properties(&self) -> PlanProperties { + self.cache.clone() + } } -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], @@ -174,6 +144,17 @@ impl MemoryExec { }) } + /// 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)))) + } + /// Set `show_sizes` to determine whether to display partition sizes pub fn with_show_sizes(mut self, show_sizes: bool) -> Self { self.show_sizes = show_sizes; @@ -237,7 +218,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 ); } @@ -259,7 +240,7 @@ 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.properties().schema())) .oeq_class .orderings; } @@ -267,7 +248,7 @@ impl MemoryExec { self.sort_information = sort_information; // We need to update equivalence properties when updating sort information. let eq_properties = EquivalenceProperties::new_with_orderings( - self.schema(), + Arc::clone(self.properties().schema()), &self.sort_information, ); self.cache = self.cache.with_eq_properties(eq_properties); @@ -375,7 +356,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, @@ -529,7 +510,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}; @@ -563,8 +545,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().to_vec(), diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 270aabeb553c..0d19120c1711 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1019,7 +1019,7 @@ mod tests { ErrorExec, MockExec, }, }, - {collect, expressions::col, memory::MemoryExec}, + {collect, expressions::col, memory::MemorySourceConfig, source::DataSourceExec}, }; use arrow::array::{ArrayRef, StringArray, UInt32Array}; @@ -1124,7 +1124,11 @@ 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 = DataSourceExec::new(Arc::new(MemorySourceConfig::try_new( + &input_partitions, + Arc::clone(schema), + None, + )?)); let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?; // execute and collect results @@ -1515,7 +1519,11 @@ 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 = DataSourceExec::new(Arc::new(MemorySourceConfig::try_new( + &input_partitions, + Arc::clone(&schema), + None, + )?)); let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?; // pull partitions @@ -1554,13 +1562,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 @@ -1599,8 +1608,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(()) @@ -1619,7 +1628,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(()) @@ -1641,8 +1650,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(()) @@ -1661,18 +1670,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 33c8a2b2fee3..f5daf0091bda 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1037,7 +1037,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}; @@ -1344,9 +1344,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 { @@ -1416,11 +1416,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()); @@ -1506,7 +1502,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 adcb28e538fd..4e1a2402dbc1 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -349,7 +349,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; @@ -415,9 +415,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)) @@ -507,9 +508,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!( @@ -695,8 +701,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()); @@ -803,9 +809,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] @@ -933,8 +937,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); @@ -975,10 +980,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); @@ -1011,8 +1016,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); @@ -1120,8 +1126,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 @@ -1231,8 +1238,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..f95196070304 --- /dev/null +++ b/datafusion/physical-plan/src/source.rs @@ -0,0 +1,139 @@ +use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; +use datafusion_common::config::ConfigOptions; +use datafusion_common::Statistics; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use std::any::Any; +use std::fmt; +use std::fmt::{Debug, Formatter}; +use std::sync::Arc; + +/// 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, + _config: &ConfigOptions, + _exec: DataSourceExec, + ) -> datafusion_common::Result>> { + Ok(None) + } + fn statistics(&self) -> datafusion_common::Result; + fn with_fetch(&self, _limit: Option) -> Option> { + None + } + fn fetch(&self) -> Option { + None + } + fn metrics(&self) -> ExecutionPlanMetricsSet; + fn properties(&self) -> PlanProperties; +} + +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>> { + self.source + .repartitioned(target_partitions, config, 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 = source.properties().clone(); + + Some(Arc::new(Self { source, cache })) + } + + fn fetch(&self) -> Option { + self.source.fetch() + } +} + +impl DataSourceExec { + pub fn new(source: Arc) -> Self { + let cache = source.properties().clone(); + Self { source, cache } + } + + /// Return the source object + #[allow(unused)] + pub fn source(&self) -> Arc { + Arc::clone(&self.source) + } + + pub fn with_source(mut self, source: Arc) -> Self { + self.cache = source.properties(); + self.source = source; + self + } +} 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 6e768a3d87bc..db7118a5592c 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -599,9 +599,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; @@ -813,14 +814,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/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 2ac86da92e50..539229d89816 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 d1fe48cfec74..d35334ab99e9 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -27,6 +27,7 @@ use object_store::ObjectMeta; use datafusion::arrow::datatypes::Schema; use datafusion::datasource::file_format::csv::CsvSink; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::file_format::json::JsonSink; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; @@ -537,6 +538,8 @@ pub fn parse_protobuf_file_scan_config( limit: proto.limit.as_ref().map(|sl| sl.limit as usize), table_partition_cols, output_ordering, + file_compression_type: FileCompressionType::UNCOMPRESSED, + new_lines_in_values: false, }) } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 292ce13d0ede..bc899eb56995 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -24,14 +24,15 @@ use prost::Message; use datafusion::arrow::compute::SortOptions; use datafusion::arrow::datatypes::SchemaRef; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::file_format::csv::CsvSink; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; 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::ParquetConfig; +use datafusion::datasource::physical_plan::{AvroConfig, CsvConfig}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::aggregate::AggregateFunctionExpr; @@ -57,12 +58,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}; @@ -203,39 +206,44 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { )), } } - PhysicalPlanType::CsvScan(scan) => Ok(Arc::new( - CsvExec::builder(parse_protobuf_file_scan_config( + PhysicalPlanType::CsvScan(scan) => { + 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 - }, - ) + )? .with_newlines_in_values(scan.newlines_in_values) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - )), + .with_file_compression_type(FileCompressionType::UNCOMPRESSED); + + 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_config = Arc::new( + CsvConfig::new( + scan.has_header, + str_to_byte(&scan.delimiter, "delimiter")?, + 0, + ) + .with_escape(escape) + .with_comment(comment), + ); + Ok(FileSourceConfig::new_exec(conf, source_config)) + } #[cfg_attr(not(feature = "parquet"), allow(unused_variables))] PhysicalPlanType::ParquetScan(scan) => { #[cfg(feature = "parquet")] @@ -257,21 +265,25 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ) }) .transpose()?; - let mut builder = ParquetExec::builder(base_config); - if let Some(predicate) = predicate { - builder = builder.with_predicate(predicate) - } - Ok(builder.build_arc()) + let source_config = Arc::new(ParquetConfig::new( + Arc::clone(&base_config.file_schema), + predicate, + None, + TableParquetOptions::default(), + )); + Ok(FileSourceConfig::new_exec(base_config, source_config)) } #[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, - )?))) + )?; + let source_config = Arc::new(AvroConfig::new()); + Ok(FileSourceConfig::new_exec(conf, source_config)) } PhysicalPlanType::CoalesceBatches(coalesce_batches) => { let input: Arc = into_physical_plan( @@ -1600,67 +1612,100 @@ 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(); + println!("DataSourceExec!"); + if let Some(maybe_csv) = source.as_any().downcast_ref::() { + let source = maybe_csv.source_config(); + println!("Source!"); + if let Some(csv_config) = source.as_any().downcast_ref::() { + println!("CSV!"); + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::CsvScan( + protobuf::CsvScanExecNode { + base_conf: Some(serialize_file_scan_config( + maybe_csv.base_config(), + 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 + .base_config() + .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, - }, - )), - }); + 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.source_config(); + 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.base_config(), + extension_codec, + )?), + predicate, + }, + )), + }); + } + } } - 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.source_config(); + 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.base_config(), + 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 0a6ea6c7ff85..4695f2fa59d7 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -39,15 +39,17 @@ use crate::cases::{ use datafusion::arrow::array::ArrayRef; use datafusion::arrow::compute::kernels::sort::SortOptions; use datafusion::arrow::datatypes::{DataType, Field, IntervalUnit, Schema}; +use datafusion::datasource::data_source::FileSourceConfig; 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, ParquetConfig, }; use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::sum::sum_udaf; @@ -723,6 +725,8 @@ 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, }; let predicate = Arc::new(BinaryExpr::new( @@ -730,11 +734,13 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { Operator::Eq, lit("1"), )); - roundtrip_test( - ParquetExec::builder(scan_config) - .with_predicate(predicate) - .build_arc(), - ) + let source_config = Arc::new(ParquetConfig::new( + Arc::clone(&scan_config.file_schema), + Some(predicate), + None, + TableParquetOptions::default(), + )); + roundtrip_test(FileSourceConfig::new_exec(scan_config, source_config)) } #[tokio::test] @@ -758,9 +764,12 @@ async fn roundtrip_parquet_exec_with_table_partition_cols() -> Result<()> { false, )], output_ordering: vec![], + file_compression_type: FileCompressionType::UNCOMPRESSED, + new_lines_in_values: false, }; + let source_config = Arc::new(ParquetConfig::default()); - roundtrip_test(ParquetExec::builder(scan_config).build_arc()) + roundtrip_test(FileSourceConfig::new_exec(scan_config, source_config)) } #[test] @@ -787,6 +796,8 @@ 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, }; #[derive(Debug, Clone, Eq)] @@ -897,9 +908,14 @@ 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 source_config = Arc::new(ParquetConfig::new( + Arc::clone(&scan_config.file_schema), + Some(custom_predicate_expr), + None, + TableParquetOptions::default(), + )); + let exec_plan = FileSourceConfig::new_exec(scan_config, source_config); 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 ced497de22a7..a1ead51aa424 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 0aedd2ad9601..7c0033da6bfa 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -148,7 +148,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 @@ -1002,7 +1002,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; @@ -4987,7 +4987,7 @@ physical_plan 09)----------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 10)------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true # @@ -5012,7 +5012,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; @@ -5036,7 +5036,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; @@ -5071,7 +5071,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; @@ -5097,7 +5097,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 @@ -5121,7 +5121,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; @@ -5145,7 +5145,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; @@ -5172,7 +5172,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; @@ -5957,7 +5957,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 @@ -5971,7 +5971,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 @@ -6044,7 +6044,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 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 8282331f995e..4ec39f769c65 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 6a63ea1cd3e4..aa74651c10b9 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 4a0ba87bfa1a..9f8e382a1780 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 cc0ebf83a843..662deaf21b2a 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 f3fee4f1fca6..0f0ecf1d6eec 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 diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 1647d61bf750..6028c3651855 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -1955,7 +1955,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 df7e21c2da44..93eff086e999 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 @@ -3431,9 +3431,9 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[sum(l.amount)] 08)--------------ProjectionExec: expr=[amount@1 as amount, sn@2 as sn, amount@3 as amount] 09)----------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +10)------------------DataSourceExec: partitions=1, partition_sizes=[1] 11)------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -12)--------------------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------------DataSourceExec: partitions=1, partition_sizes=[1] query IRR SELECT r.sn, SUM(l.amount), r.amount @@ -3580,7 +3580,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 @@ -3763,7 +3763,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) @@ -3785,7 +3785,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. @@ -3806,7 +3806,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, @@ -3840,7 +3840,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; @@ -3874,10 +3874,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 @@ -3918,7 +3918,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], 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], file_type=csv, has_header=true # drop table multiple_ordered_table_with_pk statement ok @@ -3959,7 +3959,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], 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], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -3981,7 +3981,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], 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], file_type=csv, has_header=true query TT EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb @@ -4001,7 +4001,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], 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], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4032,10 +4032,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], 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], 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], 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], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4050,7 +4050,7 @@ EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 ---- logical_plan 01)Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 -02)--Cross Join: +02)--Cross Join: 03)----SubqueryAlias: lhs 04)------Projection: multiple_ordered_table_with_pk.c, sum(multiple_ordered_table_with_pk.d) AS sum1 05)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[sum(CAST(multiple_ordered_table_with_pk.d AS Int64))]] @@ -4064,10 +4064,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], 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 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], 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], file_type=csv, has_header=true # we do not generate physical plan for Repartition yet (e.g Distribute By queries). query TT @@ -4106,10 +4106,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]], 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]], 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]], 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 # table scan should be simplified. query TT @@ -4124,7 +4124,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]], 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 # limit should be simplified query TT @@ -4143,7 +4143,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]], 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]], file_type=csv, has_header=true statement ok set datafusion.execution.target_partitions = 8; @@ -4181,7 +4181,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; @@ -4204,7 +4204,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 @@ -4292,7 +4292,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 @@ -4408,7 +4408,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 @@ -4581,7 +4581,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 @@ -4970,7 +4970,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) @@ -5115,8 +5115,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 804612287246..320f6b062c11 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 edfc2ee75bd7..88d6d0b29464 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -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 b/datafusion/sqllogictest/test_files/join.slt index 1feacc5ebe53..c88f419a9cb2 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -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 cf897d628da5..d4174738e2e0 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 68426f180d99..6c9244589f92 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 @@ -2775,12 +2775,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 @@ -2807,12 +2807,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 @@ -2869,12 +2869,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 @@ -2910,12 +2910,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 @@ -2969,10 +2969,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 @@ -3005,10 +3005,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 @@ -3066,12 +3066,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 @@ -3088,12 +3088,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 @@ -3145,10 +3145,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 @@ -3162,10 +3162,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 @@ -3247,11 +3247,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 @@ -3279,13 +3279,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; @@ -3324,14 +3324,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; @@ -3363,10 +3363,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. @@ -3390,10 +3390,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 @@ -3417,8 +3417,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. @@ -3465,10 +3465,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 @@ -3508,11 +3508,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 * @@ -3527,9 +3527,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. @@ -3546,9 +3546,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 @@ -3926,8 +3926,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] @@ -3983,8 +3983,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: @@ -4044,8 +4044,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) @@ -4150,10 +4150,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; @@ -4179,9 +4179,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 @@ -4281,8 +4281,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] # Test Utf8View as Join Key # Issue: https://github.com/apache/datafusion/issues/12468 @@ -4315,8 +4315,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 @@ -4349,12 +4349,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; @@ -4376,12 +4376,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 5b98392f1aa0..83147c64d40f 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 diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 28fc2f4b0b80..a711e17e4bf0 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -112,7 +112,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/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 a46040aa532e..0cb058395b65 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 @@ -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 diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 253ebb9ea0ac..ed239ac3cece 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 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..c0618a1a4499 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 @@ -125,7 +125,7 @@ logical_plan 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] +02)--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, string_col, bigint_col], output_ordering=[int_col@0 ASC NULLS LAST, bigint_col@2 ASC NULLS LAST], file_type=parquet # Clean up & recreate but sort on descending column statement ok @@ -157,7 +157,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 +191,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 +224,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 +259,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 878d7c8a4dfb..1939cb9c5dbe 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 800026dd766d..65c99a2b0cfe 100644 --- a/datafusion/sqllogictest/test_files/regexp.slt +++ b/datafusion/sqllogictest/test_files/regexp.slt @@ -826,7 +826,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) @@ -843,7 +843,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 630674bb09ed..663dee9b1e56 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 6f60ed8583c3..b87af6a209ad 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1849,7 +1849,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; @@ -1868,7 +1868,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 c687429ae6ec..397bb65932ff 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -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 7596b820c688..5853dbd5177f 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 25fe4c7b0390..5be25ce2ce55 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 @@ -1153,11 +1153,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 1dbce79e0f1a..d79c2edfb364 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -80,7 +80,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 c60848cb56c1..41eb92c8b572 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 \ No newline at end of file +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 \ No newline at end of file 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 80f9919cb46b..f171dd408ad1 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,15 +112,15 @@ 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 2667bb23a459..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 +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 e9486c39f08c..b9aec6b61317 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -84,17 +84,17 @@ 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_headerr=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_headerr=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_headerr=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index f28a5ef54885..26b5a045c588 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -61,17 +61,17 @@ 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 e3918ba62b02..99b2b79bad14 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,5 +123,5 @@ 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..170694149f8f 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_headerr=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_headerr=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_headerr=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_headerr=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_headerr=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_headerr=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_headerr=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_headerr=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 7b8992b966ad..b199cc48ef63 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 Int64) 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 Int64) 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 TI 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 @@ -525,7 +525,7 @@ physical_plan 13)------------------------CoalesceBatchesExec: target_batch_size=2 14)--------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] 15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +16)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true 17)------ProjectionExec: expr=[1 as cnt] 18)--------PlaceholderRowExec 19)------ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] @@ -610,9 +610,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 @@ -827,11 +827,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 2685e18427ca..b2c741d4a698 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 56f088dfd10f..81a177fd6218 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)--ProjectionExec: expr=[c9@0 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] 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 diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index f4548df4309e..b88ab0a0817c 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -20,9 +20,10 @@ use std::sync::Arc; use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::common::{not_impl_err, substrait_err}; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; @@ -150,7 +151,8 @@ pub async fn from_substrait_rel( } } - Ok(ParquetExec::builder(base_config).build_arc() + let source_config = Arc::new(ParquetConfig::default()); + Ok(FileSourceConfig::new_exec(base_config, source_config) as Arc) } _ => not_impl_err!( diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 7279785ae873..086847c33431 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -15,9 +15,15 @@ // specific language governing permissions and limitations // under the License. +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::datasource::data_source::FileSourceConfig; +use datafusion::datasource::physical_plan::ParquetConfig; use datafusion::error::{DataFusionError, Result}; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::{displayable, ExecutionPlan}; use std::collections::HashMap; use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; @@ -35,11 +41,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 +49,97 @@ 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 + .source_config() + .as_any() + .downcast_ref::() + .is_some(); + if is_parquet { + let base_config = file_config.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 {})), + }); + } + } - 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 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); + } + 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) = 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(); - 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..26fc90a394ef 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -20,9 +20,10 @@ use std::sync::Arc; use datafusion::arrow::datatypes::Schema; use datafusion::dataframe::DataFrame; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use datafusion::error::Result; use datafusion::physical_plan::{displayable, ExecutionPlan}; use datafusion::prelude::{ParquetReadOptions, SessionContext}; @@ -46,8 +47,9 @@ async fn parquet_exec() -> Result<()> { 123, )], ]); + let source_config = Arc::new(ParquetConfig::default()); let parquet_exec: Arc = - ParquetExec::builder(scan_config).build_arc(); + FileSourceConfig::new_exec(scan_config, source_config); 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/user-guide/explain-usage.md b/docs/source/user-guide/explain-usage.md index 2eb03aad2ef9..f68c309c2289 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 = ...) │ └────────────────────────────────────────────────┘ ``` @@ -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. @@ -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 | | | | +---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ``` 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=[] | +-------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------+ ``` From 70505b333eed9a788bf2fcf47445d24d71dc7cba Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 7 Jan 2025 17:04:32 +0300 Subject: [PATCH 02/39] fix license headers --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 1 + datafusion/physical-plan/src/source.rs | 17 +++++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 2a34a9d4ea32..17bf18b8dccf 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -14,6 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. + use std::sync::Arc; use std::time::SystemTime; diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs index f95196070304..ebfc17321b22 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/physical-plan/src/source.rs @@ -1,3 +1,20 @@ +// 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 crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; use datafusion_common::config::ConfigOptions; From 00db75948c63a92f91286352e832a15173801deb Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 7 Jan 2025 17:34:34 +0300 Subject: [PATCH 03/39] fix compile errors on documents --- .../core/src/datasource/physical_plan/csv.rs | 3 ++- .../src/datasource/physical_plan/parquet/mod.rs | 14 ++++++++------ 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index d9ff46456c1f..77aeae4298e9 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -52,6 +52,7 @@ use tokio::task::JoinSet; /// # physical_plan::FileScanConfig, /// # listing::PartitionedFile, /// # }; +/// use datafusion::datasource::data_source::FileSourceConfig; /// # use datafusion::datasource::physical_plan::CsvConfig; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_physical_plan::source::DataSourceExec; @@ -70,7 +71,7 @@ use tokio::task::JoinSet; /// ) /// .with_terminator(Some(b'#') /// )); -/// let exec = DataSourceExec::builder(file_scan_config, source_config); +/// let exec = FileSourceConfig::new_exec(file_scan_config, source_config); /// ``` #[derive(Debug, Clone)] pub struct CsvConfig { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 1cb81c76d2c2..25c8a7f95622 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -183,18 +183,20 @@ pub use writer::plan_to_parquet; /// /// # fn parquet_exec() -> DataSourceExec { unimplemented!() } /// // Split a single DataSourceExec into multiple DataSourceExecs, one for each file -/// let exec = parquet_exec(); -/// let data_source = exec.source(); -/// let existing_file_groups = &data_source.base_config().file_groups; +/// let source = parquet_exec().source(); +/// let data_source = source.as_any().downcast_ref::().unwrap(); +/// let base_config = data_source.base_config(); +/// let source_config = data_source.source_config(); +/// 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 = data_source -/// .base_config() +/// let new_config = base_config +/// .clone() /// .with_file_groups(vec![file_group.clone()]); /// -/// DataSourceExec::new(Arc::new(FileSourceConfig::new(new_config, data_source.source_config()))) +/// DataSourceExec::new(Arc::new(FileSourceConfig::new(new_config, source_config.clone()))) /// }) /// .collect::>(); /// ``` From cea3ecde590d1a3515dd2f118b4de2d8bb3d913a Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 7 Jan 2025 17:52:12 +0300 Subject: [PATCH 04/39] separate non-parquet code --- datafusion/core/src/datasource/data_source.rs | 119 ++++++++++-------- 1 file changed, 64 insertions(+), 55 deletions(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 54a09c8ce433..29900d819d90 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -25,8 +25,10 @@ use std::sync::Arc; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{ ArrowConfig, AvroConfig, CsvConfig, FileGroupPartitioner, FileOpener, FileScanConfig, - FileStream, JsonConfig, ParquetConfig, + FileStream, JsonConfig, }; +#[cfg(feature = "parquet")] +use crate::datasource::physical_plan::ParquetConfig; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; @@ -114,40 +116,44 @@ impl DataSource for FileSourceConfig { self.fmt_source_config(f)?; if let Some(csv_conf) = self.source_config.as_any().downcast_ref::() { - write!(f, ", has_header={}", csv_conf.has_header) - } else if let Some(parquet_conf) = - self.source_config.as_any().downcast_ref::() - { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - let predicate_string = parquet_conf - .predicate() - .map(|p| format!(", predicate={p}")) - .unwrap_or_default(); - - let pruning_predicate_string = parquet_conf - .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,) + return write!(f, ", has_header={}", csv_conf.has_header) + } + + #[cfg(feature = "parquet")] + if cfg!(feature = "parquet") { + if let Some(parquet_conf) = + self.source_config.as_any().downcast_ref::() + { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let predicate_string = parquet_conf + .predicate() + .map(|p| format!(", predicate={p}")) + .unwrap_or_default(); + + let pruning_predicate_string = parquet_conf + .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(); + + return write!(f, "{}{}", predicate_string, pruning_predicate_string,) + } } } - } else { - Ok(()) } + Ok(()) } /// Redistribute files across partitions according to their size @@ -179,24 +185,28 @@ impl DataSource for FileSourceConfig { } fn statistics(&self) -> datafusion_common::Result { - let stats = if let Some(parquet_config) = - self.source_config.as_any().downcast_ref::() - { - // 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 parquet_config.pruning_predicate().is_some() - || parquet_config.page_pruning_predicate().is_some() - || (parquet_config.predicate().is_some() + + #[cfg(not(feature = "parquet"))] + let stats = self.projected_statistics.clone(); + + #[cfg(feature = "parquet")] + let stats = + if let Some(parquet_config) = self.source_config.as_any().downcast_ref::() { + // 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 parquet_config.pruning_predicate().is_some() + || parquet_config.page_pruning_predicate().is_some() + || (parquet_config.predicate().is_some() && parquet_config.pushdown_filters()) - { - self.projected_statistics.clone().to_inexact() + { + self.projected_statistics.clone().to_inexact() + } else { + self.projected_statistics.clone() + } } else { self.projected_statistics.clone() - } - } else { - self.projected_statistics.clone() - }; + }; Ok(stats) } @@ -250,6 +260,7 @@ impl FileSourceConfig { ); let mut metrics = ExecutionPlanMetricsSet::new(); + #[cfg(feature = "parquet")] if let Some(parquet_config) = source_config.as_any().downcast_ref::() { @@ -266,9 +277,10 @@ impl FileSourceConfig { cache, } } + /// Write the data_type based on source_config fn fmt_source_config(&self, f: &mut Formatter) -> fmt::Result { - let data_type = if self + let mut data_type = if self .source_config .as_any() .downcast_ref::() @@ -296,16 +308,13 @@ impl FileSourceConfig { .is_some() { "json" - } else if self - .source_config - .as_any() - .downcast_ref::() - .is_some() - { - "parquet" } else { "unknown" }; + #[cfg(feature = "parquet")] + if self.source_config.as_any().downcast_ref::().is_some() { + data_type = "parquet"; + } write!(f, ", file_type={}", data_type) } From 233157756bf4e2f955607ce9fb195b7869fb9349 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 8 Jan 2025 09:25:11 +0300 Subject: [PATCH 05/39] format code --- datafusion/core/src/datasource/data_source.rs | 49 +++++++++++-------- 1 file changed, 29 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 29900d819d90..4f259d992db1 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -23,12 +23,12 @@ use std::fmt::Formatter; use std::sync::Arc; use crate::datasource::listing::PartitionedFile; +#[cfg(feature = "parquet")] +use crate::datasource::physical_plan::ParquetConfig; use crate::datasource::physical_plan::{ ArrowConfig, AvroConfig, CsvConfig, FileGroupPartitioner, FileOpener, FileScanConfig, FileStream, JsonConfig, }; -#[cfg(feature = "parquet")] -use crate::datasource::physical_plan::ParquetConfig; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; @@ -116,7 +116,7 @@ impl DataSource for FileSourceConfig { self.fmt_source_config(f)?; if let Some(csv_conf) = self.source_config.as_any().downcast_ref::() { - return write!(f, ", has_header={}", csv_conf.has_header) + return write!(f, ", has_header={}", csv_conf.has_header); } #[cfg(feature = "parquet")] @@ -148,7 +148,11 @@ impl DataSource for FileSourceConfig { }) .unwrap_or_default(); - return write!(f, "{}{}", predicate_string, pruning_predicate_string,) + return write!( + f, + "{}{}", + predicate_string, pruning_predicate_string, + ); } } } @@ -185,28 +189,28 @@ impl DataSource for FileSourceConfig { } fn statistics(&self) -> datafusion_common::Result { - #[cfg(not(feature = "parquet"))] let stats = self.projected_statistics.clone(); #[cfg(feature = "parquet")] - let stats = - if let Some(parquet_config) = self.source_config.as_any().downcast_ref::() { - // 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 parquet_config.pruning_predicate().is_some() - || parquet_config.page_pruning_predicate().is_some() - || (parquet_config.predicate().is_some() + let stats = if let Some(parquet_config) = + self.source_config.as_any().downcast_ref::() + { + // 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 parquet_config.pruning_predicate().is_some() + || parquet_config.page_pruning_predicate().is_some() + || (parquet_config.predicate().is_some() && parquet_config.pushdown_filters()) - { - self.projected_statistics.clone().to_inexact() - } else { - self.projected_statistics.clone() - } + { + self.projected_statistics.clone().to_inexact() } else { self.projected_statistics.clone() - }; + } + } else { + self.projected_statistics.clone() + }; Ok(stats) } @@ -312,7 +316,12 @@ impl FileSourceConfig { "unknown" }; #[cfg(feature = "parquet")] - if self.source_config.as_any().downcast_ref::().is_some() { + if self + .source_config + .as_any() + .downcast_ref::() + .is_some() + { data_type = "parquet"; } write!(f, ", file_type={}", data_type) From 15b812f61fb0bd4a48295013a75b9f6a0de1b536 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 8 Jan 2025 09:32:32 +0300 Subject: [PATCH 06/39] fix typo --- .../test_files/tpch/plans/q16.slt.part | 6 +++--- .../test_files/tpch/plans/q8.slt.part | 16 ++++++++-------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index b9aec6b61317..7cb60ad0a7b2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -84,17 +84,17 @@ 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)--------------------------------------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_headerr=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)--------------------------------------------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_headerr=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)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], file_type=csv, has_headerr=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/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 170694149f8f..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)------------------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], file_type=csv, has_headerr=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)------------------------------------------------------------------------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_headerr=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)------------------------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_headerr=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)------------------------------------------------------------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_headerr=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)--------------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], file_type=csv, has_headerr=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)------------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_headerr=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)----------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], file_type=csv, has_headerr=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)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_headerr=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 From f020bd2dda138ca3604cf7fabb56b2d872cc292d Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 8 Jan 2025 09:37:28 +0300 Subject: [PATCH 07/39] fix imports --- datafusion-cli/src/functions.rs | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index 6160d6b07282..9594efe9d440 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -16,29 +16,33 @@ // 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; use datafusion::common::{plan_err, Column}; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::logical_expr::Expr; +use datafusion::physical_plan::memory::MemorySourceConfig; +use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::scalar::ScalarValue; use datafusion_catalog::TableFunctionImpl; + +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 { From a7a5bd86a5e8f08fcc91231a8fba2ede2efc458e Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 8 Jan 2025 11:03:21 +0300 Subject: [PATCH 08/39] fix clippy fix csv_json example --- .../examples/csv_json_opener.rs | 27 ++++++++++--------- .../core/src/datasource/file_format/arrow.rs | 6 ++--- .../limited_distinct_aggregation.rs | 6 +---- .../src/aggregate_statistics.rs | 6 +---- 4 files changed, 18 insertions(+), 27 deletions(-) diff --git a/datafusion-examples/examples/csv_json_opener.rs b/datafusion-examples/examples/csv_json_opener.rs index ba18e84b0ec4..737e1413580c 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -18,13 +18,14 @@ use std::sync::Arc; use arrow_schema::{DataType, Field, Schema}; +use datafusion::datasource::data_source::DataSourceFileConfig; use datafusion::{ assert_batches_eq, datasource::{ file_format::file_compression_type::FileCompressionType, listing::PartitionedFile, object_store::ObjectStoreUrl, - physical_plan::{CsvConfig, CsvOpener, FileScanConfig, FileStream, JsonOpener}, + physical_plan::{CsvConfig, FileScanConfig, FileStream, JsonOpener}, }, error::Result, physical_plan::metrics::ExecutionPlanMetricsSet, @@ -48,23 +49,23 @@ async fn csv_opener() -> Result<()> { let object_store = Arc::new(LocalFileSystem::new()); let schema = aggr_test_schema(); - let config = CsvConfig::new(true, b',', b'"').with_terminator(Some(b'#')); - - let opener = Arc::new(CsvOpener::new( - Arc::new(config), - FileCompressionType::UNCOMPRESSED, - object_store, - )); - 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)) + .with_projection(Some(vec![12, 0])) + .with_limit(Some(5)) + .with_file(PartitionedFile::new(path.display().to_string(), 10)); + + let config = CsvConfig::new(true, b',', 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 = diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index d91c8d2dde20..6963cace03fa 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -53,7 +53,6 @@ use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::metrics::MetricsSet; -use datafusion_physical_plan::source::DataSourceExec; use async_trait::async_trait; use bytes::Bytes; @@ -174,9 +173,8 @@ impl FileFormat for ArrowFormat { _filters: Option<&Arc>, ) -> Result> { let source_config = Arc::new(ArrowConfig {}); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let exec = DataSourceExec::new(source); - Ok(Arc::new(exec)) + let exec = FileSourceConfig::new_exec(conf, source_config); + Ok(exec) } async fn create_writer_physical_plan( diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 00570468e023..1f2ab55d2b2b 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -78,11 +78,7 @@ fn mock_data() -> Result> { ], )?; - Ok(MemorySourceConfig::try_new_exec( - &[vec![batch]], - Arc::clone(&schema), - None, - )?) + MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) } fn assert_plan_matches_expected( diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index b0235d920e9c..9a354e8a1eb4 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -249,11 +249,7 @@ mod tests { ], )?; - Ok(MemorySourceConfig::try_new_exec( - &[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 From d69b01265229d59d38fa43e4722184b6e629c569 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 8 Jan 2025 11:07:59 +0300 Subject: [PATCH 09/39] add comment to the example --- datafusion-examples/examples/csv_json_opener.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion-examples/examples/csv_json_opener.rs b/datafusion-examples/examples/csv_json_opener.rs index 737e1413580c..9f15c710381a 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -61,6 +61,7 @@ async fn csv_opener() -> Result<()> { .with_file(PartitionedFile::new(path.display().to_string(), 10)); let config = CsvConfig::new(true, b',', b'"') + .with_comment(Some(b'#')) .with_schema(schema) .with_batch_size(8192) .with_projection(&scan_config); From f14a6d8016d05a1da085a6ff2243ebe727c1a970 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 8 Jan 2025 11:17:44 +0300 Subject: [PATCH 10/39] fix cargo docs --- datafusion/core/src/datasource/physical_plan/arrow_file.rs | 2 +- datafusion/core/src/datasource/physical_plan/avro.rs | 2 +- datafusion/core/src/datasource/physical_plan/parquet/mod.rs | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index b952b832cc4a..81fe134c5e14 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -33,7 +33,7 @@ use futures::StreamExt; use itertools::Itertools; use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; -/// Arrow configuration struct that is given to [`DataSourceExec`] +/// Arrow configuration struct that is given to DataSourceExec /// Does not hold anything special, since [`FileScanConfig`] is sufficient for arrow #[derive(Clone, Default)] pub struct ArrowConfig {} diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 5b3f49e691d6..28851b61d83b 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -30,7 +30,7 @@ use arrow::datatypes::SchemaRef; use object_store::ObjectStore; -/// AvroConfig holds the extra configuration that is necessary for [`AvroOpener`] +/// AvroConfig holds the extra configuration that is necessary for opening avro files #[derive(Clone, Default)] pub struct AvroConfig { schema: Option, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 25c8a7f95622..5065ee87fdd2 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 +//! [`ParquetConfig`] FileSourceConfig for reading Parquet files use std::any::Any; use std::sync::Arc; @@ -205,7 +205,7 @@ pub use writer::plan_to_parquet; /// /// 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 +/// [`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. /// @@ -250,7 +250,7 @@ pub use writer::plan_to_parquet; /// /// # Execution Overview /// -/// * Step 1: [`DataSourceExec::execute`] is called, returning a [`FileStream`] +/// * 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 From 804014757b2c4b7cae7ec710f386b63e93a0153c Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 8 Jan 2025 11:36:40 +0300 Subject: [PATCH 11/39] change MemoryExec with MemorySourceConfig --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 41e4aa273827..1c6b9d071e44 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2807,7 +2807,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())], From ddb221d53c37e4a5ae0da302963a92823fb3e8b0 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 10 Jan 2025 14:00:37 +0300 Subject: [PATCH 12/39] merge fixes --- .../replace_with_order_preserving_variants.rs | 21 ++++++++++--------- datafusion/physical-plan/src/memory.rs | 4 ++-- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index d38a3bb51095..67689003f9de 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -285,10 +285,6 @@ pub(crate) fn replace_with_order_preserving_variants( mod tests { use super::*; - use crate::datasource::data_source::FileSourceConfig; - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::physical_plan::{CsvConfig, FileScanConfig}; use crate::execution::TaskContext; use crate::physical_optimizer::test_utils::check_integrity; use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; @@ -311,12 +307,14 @@ mod tests { use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::collect; + use datafusion_physical_plan::memory::MemorySourceConfig; + use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::streaming::StreamingTableExec; + use object_store::memory::InMemory; use object_store::ObjectStore; - use url::Url; - use rstest::rstest; + use url::Url; /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts /// the plan against the original and expected plans for both bounded and @@ -1569,10 +1567,13 @@ mod tests { .map(|_| vec![make_partition(schema, rows)]) .collect(); let projection: Vec = vec![0, 2, 3]; - DataSourceExec::try_new(&data, schema.clone(), Some(projection)) - .unwrap() - .try_with_sort_information(vec![sort_exprs]) - .unwrap() + let source = Arc::new( + MemorySourceConfig::try_new(&data, schema.clone(), Some(projection)) + .unwrap() + .try_with_sort_information(vec![sort_exprs]) + .unwrap(), + ); + DataSourceExec::new(source) }) } } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 0aa4205c605c..99c6426f86f5 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -241,8 +241,8 @@ impl MemorySourceConfig { ProjectionMapping::try_new(&proj_exprs, &self.original_schema())?; sort_information = base_eqp .project(&projection_mapping, Arc::clone(self.properties().schema())) - .into_oeq_class - .into_inner; + .into_oeq_class() + .into_inner(); } self.sort_information = sort_information; From 069d28cc7a404c6b6de378c74f1fc1c35ebc4546 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Fri, 10 Jan 2025 17:36:38 +0300 Subject: [PATCH 13/39] change MemoryExec to DataSourceExec --- datafusion/sqllogictest/test_files/window.slt | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 444024f52910..91c4213b9001 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -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 From cb6a5ffcacffcca03ed743fc08959128399b9ef4 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 15 Jan 2025 14:26:32 +0300 Subject: [PATCH 14/39] fix merge conflicts --- datafusion/core/src/datasource/file_format/json.rs | 4 +++- datafusion/core/src/datasource/file_format/parquet.rs | 8 ++++---- datafusion/core/src/physical_planner.rs | 5 ++--- datafusion/physical-plan/src/memory.rs | 10 +++++++--- 4 files changed, 16 insertions(+), 11 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 8948bf6345a9..950c86df71ba 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -33,7 +33,9 @@ use crate::datasource::data_source::FileSourceConfig; 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, FileSinkConfig, JsonConfig}; +use crate::datasource::physical_plan::{ + FileGroupDisplay, FileSink, FileSinkConfig, JsonConfig, +}; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::insert::{DataSink, DataSinkExec}; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 775b9c440ae3..917418d02aba 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -32,12 +32,13 @@ use super::{ }; use crate::arrow::array::RecordBatch; use crate::arrow::datatypes::{Fields, Schema, SchemaRef}; +use crate::datasource::data_source::FileSourceConfig; 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, +use crate::datasource::physical_plan::parquet::can_expr_be_pushed_down_with_schemas; +use crate::datasource::physical_plan::{ + FileGroupDisplay, FileSink, FileSinkConfig, ParquetConfig, }; -use crate::datasource::physical_plan::{FileGroupDisplay, FileSink, FileSinkConfig}; use crate::datasource::statistics::{create_max_min_accs, get_col_stats}; use crate::error::Result; use crate::execution::context::SessionState; @@ -46,7 +47,6 @@ use crate::physical_plan::{ Accumulator, DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, Statistics, }; -use crate::datasource::data_source::FileSourceConfig; use arrow::compute::sum; use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions}; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 7664d1287090..c89b4b2aa5da 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -466,9 +466,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, diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index dd5db6278a7f..58800ecf72b6 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -816,7 +816,8 @@ mod tests { Field::new("col0", DataType::UInt32, false), Field::new("col1", DataType::Utf8, false), ])); - let _ = MemorySourceConfig::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 @@ -827,8 +828,11 @@ mod tests { DataType::UInt32, false, )])); - let _ = MemorySourceConfig::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 _ = MemorySourceConfig::try_new_as_values( schema, From 4acb2fc459ee9b7176be3922a35ea1add6f0b363 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 15 Jan 2025 15:45:39 +0300 Subject: [PATCH 15/39] apply some syntactic sugars --- datafusion-cli/src/functions.rs | 7 ++--- .../examples/advanced_parquet_index.rs | 5 ++- datafusion-examples/examples/parquet_index.rs | 5 ++- .../examples/remote_catalog.rs | 7 ++--- datafusion-examples/examples/simple_udtf.rs | 7 ++--- datafusion/core/benches/physical_plan.rs | 18 +++++------ datafusion/core/benches/sort.rs | 26 +++++----------- .../core/src/datasource/file_format/avro.rs | 6 ++-- .../core/src/datasource/file_format/csv.rs | 6 ++-- .../core/src/datasource/file_format/json.rs | 6 ++-- .../src/datasource/file_format/parquet.rs | 6 ++-- datafusion/core/src/datasource/memory.rs | 9 ++---- .../core/src/datasource/physical_plan/avro.rs | 11 ++----- .../core/src/datasource/physical_plan/csv.rs | 19 ++++-------- .../core/src/datasource/physical_plan/json.rs | 13 +++----- .../datasource/physical_plan/parquet/mod.rs | 31 ++++++++----------- .../core/src/datasource/schema_adapter.rs | 6 ++-- .../physical_optimizer/projection_pushdown.rs | 11 +++---- .../core/src/physical_optimizer/test_utils.rs | 4 +-- datafusion/core/src/physical_planner.rs | 9 ++---- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 11 ++++--- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 17 +++++----- .../core/tests/fuzz_cases/merge_fuzz.rs | 6 ++-- datafusion/core/tests/fuzz_cases/pruning.rs | 4 +-- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 6 ++-- .../core/tests/fuzz_cases/window_fuzz.rs | 8 ++--- .../core/tests/parquet/custom_reader.rs | 6 ++-- .../core/tests/parquet/schema_coercion.rs | 13 ++++---- .../physical-plan/src/repartition/mod.rs | 14 ++++----- .../test_files/insert_to_external.slt | 2 +- datafusion/sqllogictest/test_files/order.slt | 4 +-- datafusion/sqllogictest/test_files/select.slt | 6 ++-- 32 files changed, 114 insertions(+), 195 deletions(-) diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index 9594efe9d440..212ddada4ace 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -32,7 +32,6 @@ use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::logical_expr::Expr; use datafusion::physical_plan::memory::MemorySourceConfig; -use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::scalar::ScalarValue; use datafusion_catalog::TableFunctionImpl; @@ -244,12 +243,12 @@ impl TableProvider for ParquetMetadataTable { _filters: &[Expr], _limit: Option, ) -> Result> { - let source = Arc::new(MemorySourceConfig::try_new( + let exec = MemorySourceConfig::try_new_exec( &[vec![self.batch.clone()]], TableProvider::schema(self), projection.cloned(), - )?); - Ok(Arc::new(DataSourceExec::new(source))) + )?; + Ok(exec) } } diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 841dec90f0b9..9cd50138f3b2 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -45,7 +45,6 @@ use datafusion::parquet::schema::types::ColumnPath; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_optimizer::pruning::PruningPredicate; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; -use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; use datafusion_common::config::TableParquetOptions; @@ -509,9 +508,9 @@ impl TableProvider for IndexTableProvider { // provide the factory to create parquet reader without re-reading metadata .with_parquet_file_reader_factory(Arc::new(reader_factory)), ); - let source = Arc::new(FileSourceConfig::new(file_scan_config, source_config)); // Finally, put it all together into a DataSourceExec - Ok(Arc::new(DataSourceExec::new(source))) + let exec = FileSourceConfig::new_exec(file_scan_config, source_config); + Ok(exec) } /// Tell DataFusion to push filters down to the scan method diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index ec31a1bc3c2b..1d11df49798f 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -34,7 +34,6 @@ use datafusion::parquet::arrow::{ arrow_reader::ParquetRecordBatchReaderBuilder, ArrowWriter, }; use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; -use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; use datafusion_common::config::TableParquetOptions; @@ -264,8 +263,8 @@ impl TableProvider for IndexTableProvider { None, TableParquetOptions::default(), )); - let source = Arc::new(FileSourceConfig::new(file_scan_config, source_config)); - Ok(Arc::new(DataSourceExec::new(source))) + let exec = FileSourceConfig::new_exec(file_scan_config, source_config); + Ok(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 8814636dfabc..60dd5944e799 100644 --- a/datafusion-examples/examples/remote_catalog.rs +++ b/datafusion-examples/examples/remote_catalog.rs @@ -36,7 +36,6 @@ use datafusion::catalog::TableProvider; use datafusion::common::Result; use datafusion::execution::SendableRecordBatchStream; use datafusion::physical_plan::memory::MemorySourceConfig; -use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{DataFrame, SessionContext}; @@ -255,11 +254,11 @@ impl TableProvider for RemoteTable { .await? .try_collect() .await?; - let source = Arc::new(MemorySourceConfig::try_new( + let exec = MemorySourceConfig::try_new_exec( &[batches], self.schema.clone(), projection.cloned(), - )?); - Ok(Arc::new(DataSourceExec::new(source))) + )?; + Ok(exec) } } diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index 47ab92349982..bc6ff32e6655 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -25,7 +25,6 @@ use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::execution::context::ExecutionProps; use datafusion::physical_plan::memory::MemorySourceConfig; -use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; use datafusion_catalog::TableFunctionImpl; @@ -121,12 +120,12 @@ impl TableProvider for LocalCsvTable { } else { self.batches.clone() }; - let source = Arc::new(MemorySourceConfig::try_new( + let exec = MemorySourceConfig::try_new_exec( &[batches], TableProvider::schema(self), projection.cloned(), - )?); - Ok(Arc::new(DataSourceExec::new(source))) + )?; + Ok(exec) } } diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index f30a35503447..a75de235d1dc 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -37,7 +37,6 @@ use datafusion::physical_plan::{ use datafusion::prelude::SessionContext; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; // Initialize the operator using the provided record batches and the sort key // as inputs. All record batches must have the same schema. @@ -56,16 +55,13 @@ fn sort_preserving_merge_operator( }) .collect::(); - let source = Arc::new( - MemorySourceConfig::try_new( - &batches.into_iter().map(|rb| vec![rb]).collect::>(), - schema, - None, - ) - .unwrap(), - ); - let exec = DataSourceExec::new(source); - let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + let exec = MemorySourceConfig::try_new_exec( + &batches.into_iter().map(|rb| vec![rb]).collect::>(), + schema, + None, + ) + .unwrap(); + 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 bfa1f7a34001..0e2280c8795b 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -88,7 +88,6 @@ use datafusion::{ use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; /// Benchmarks for SortPreservingMerge stream use criterion::{criterion_group, criterion_main, Criterion}; @@ -169,10 +168,8 @@ impl BenchCase { let schema = partitions[0][0].schema(); let sort = make_sort_exprs(schema.as_ref()); - let source = - Arc::new(MemorySourceConfig::try_new(partitions, schema, None).unwrap()); - let exec = DataSourceExec::new(source); - 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, @@ -190,11 +187,8 @@ impl BenchCase { let schema = partitions[0][0].schema(); let sort = make_sort_exprs(schema.as_ref()); - let source = - Arc::new(MemorySourceConfig::try_new(partitions, schema, None).unwrap()); - let exec = DataSourceExec::new(source); - 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 { @@ -214,10 +208,8 @@ impl BenchCase { let schema = partitions[0][0].schema(); let sort = make_sort_exprs(schema.as_ref()); - let source = - Arc::new(MemorySourceConfig::try_new(partitions, schema, None).unwrap()); - let exec = DataSourceExec::new(source); - 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 { @@ -237,10 +229,8 @@ impl BenchCase { let schema = partitions[0][0].schema(); let sort = make_sort_exprs(schema.as_ref()); - let source = - Arc::new(MemorySourceConfig::try_new(partitions, schema, None).unwrap()); - let exec = DataSourceExec::new(source); - 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/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index 178b58e24b18..bde69da39f98 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -41,7 +41,6 @@ use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::GetExt; use datafusion_common::DEFAULT_AVRO_EXTENSION; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::source::DataSourceExec; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; #[derive(Default)] @@ -153,9 +152,8 @@ impl FileFormat for AvroFormat { _filters: Option<&Arc>, ) -> Result> { let source_config = Arc::new(AvroConfig::new()); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let exec = DataSourceExec::new(source); - Ok(Arc::new(exec)) + let exec = FileSourceConfig::new_exec(conf, source_config); + Ok(exec) } } diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index f64844548596..8508b213f90f 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -55,7 +55,6 @@ use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexRequirement; -use datafusion_physical_plan::source::DataSourceExec; use async_trait::async_trait; use bytes::{Buf, Bytes}; @@ -434,9 +433,8 @@ impl FileFormat for CsvFormat { .with_comment(self.options.comment), ); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let exec = DataSourceExec::new(source); - Ok(Arc::new(exec)) + let exec = FileSourceConfig::new_exec(conf, source_config); + Ok(exec) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 950c86df71ba..893d9dd8eede 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -56,7 +56,6 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::ExecutionPlan; use async_trait::async_trait; @@ -255,9 +254,8 @@ impl FileFormat for JsonFormat { let source_config = Arc::new(JsonConfig::new()); conf.file_compression_type = FileCompressionType::from(self.options.compression); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let exec = DataSourceExec::new(source); - Ok(Arc::new(exec)) + let exec = FileSourceConfig::new_exec(conf, source_config); + Ok(exec) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 917418d02aba..92213bf31bed 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -65,7 +65,6 @@ use datafusion_expr::Expr; use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexRequirement; -use datafusion_physical_plan::source::DataSourceExec; use async_trait::async_trait; use bytes::Bytes; @@ -424,9 +423,8 @@ impl FileFormat for ParquetFormat { metadata_size_hint, self.options.clone(), )); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - - Ok(Arc::new(DataSourceExec::new(source))) + let exec = FileSourceConfig::new_exec(conf, source_config); + Ok(exec) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 35ea0a19411d..1a4fd884f3fd 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -162,16 +162,11 @@ impl MemTable { } } - let source = Arc::new(MemorySourceConfig::try_new( - &data, - Arc::clone(&schema), - None, - )?); - let exec = DataSourceExec::new(source); + let exec = MemorySourceConfig::try_new_exec(&data, Arc::clone(&schema), None)?; if let Some(num_partitions) = output_partitions { let exec = RepartitionExec::try_new( - Arc::new(exec), + exec, Partitioning::RoundRobinBatch(num_partitions), )?; diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 28851b61d83b..0d3daa0ac491 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -152,7 +152,6 @@ mod tests { use crate::scalar::ScalarValue; use crate::test::object_store::local_unpartitioned_file; - use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::ExecutionPlan; use futures::StreamExt; @@ -199,9 +198,7 @@ mod tests { .with_projection(Some(vec![0, 1, 2])); let source_config = Arc::new(AvroConfig::new()); - - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let source_exec = DataSourceExec::new(source); + let source_exec = FileSourceConfig::new_exec(conf, source_config); assert_eq!( source_exec .properties() @@ -274,8 +271,7 @@ mod tests { .with_projection(projection); let source_config = Arc::new(AvroConfig::new()); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let source_exec = DataSourceExec::new(source); + let source_exec = FileSourceConfig::new_exec(conf, source_config); assert_eq!( source_exec .properties() @@ -349,8 +345,7 @@ mod tests { .with_table_partition_cols(vec![Field::new("date", DataType::Utf8, false)]); let source_config = Arc::new(AvroConfig::new()); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let source_exec = DataSourceExec::new(source); + let source_exec = FileSourceConfig::new_exec(conf, source_config); assert_eq!( source_exec diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 77aeae4298e9..b46f2d8684c1 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -423,7 +423,6 @@ mod tests { use crate::datasource::data_source::FileSourceConfig; use datafusion_physical_plan::metrics::MetricsSet; - use datafusion_physical_plan::source::DataSourceExec; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; @@ -467,8 +466,7 @@ mod tests { config.projection = Some(vec![0, 2, 4]); let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); - let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); - let csv = Arc::new(DataSourceExec::new(source)); + let csv = FileSourceConfig::new_exec(config.clone(), source_config); assert_eq!(13, config.file_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); @@ -533,8 +531,7 @@ mod tests { config.projection = Some(vec![4, 0, 2]); let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); - let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); - let csv = Arc::new(DataSourceExec::new(source)); + let csv = FileSourceConfig::new_exec(config.clone(), source_config); assert_eq!(13, config.file_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); @@ -598,8 +595,7 @@ mod tests { config.limit = Some(5); let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); - let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); - let csv = Arc::new(DataSourceExec::new(source)); + let csv = FileSourceConfig::new_exec(config.clone(), source_config); assert_eq!(13, config.file_schema.fields().len()); assert_eq!(13, csv.schema().fields().len()); @@ -660,8 +656,7 @@ mod tests { config.limit = Some(5); let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); - let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); - let csv = Arc::new(DataSourceExec::new(source)); + let csv = FileSourceConfig::new_exec(config.clone(), source_config); assert_eq!(14, config.file_schema.fields().len()); assert_eq!(14, csv.schema().fields().len()); @@ -722,8 +717,7 @@ mod tests { // partitions are resolved during scan anyway let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); - let source = Arc::new(FileSourceConfig::new(config.clone(), source_config)); - let csv = Arc::new(DataSourceExec::new(source)); + let csv = FileSourceConfig::new_exec(config.clone(), source_config); assert_eq!(13, config.file_schema.fields().len()); assert_eq!(2, csv.schema().fields().len()); @@ -813,8 +807,7 @@ mod tests { .with_newlines_in_values(false) .with_file_compression_type(file_compression_type.to_owned()); let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); - let source = Arc::new(FileSourceConfig::new(config, source_config)); - let csv = Arc::new(DataSourceExec::new(source)); + let csv = FileSourceConfig::new_exec(config.clone(), source_config); let it = csv.execute(0, task_ctx).unwrap(); let batches: Vec<_> = it.try_collect().await.unwrap(); diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 70a872d8b849..349eceadddc0 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -259,7 +259,6 @@ 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 datafusion_physical_plan::source::DataSourceExec; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; @@ -390,8 +389,7 @@ mod tests { let source_config = Arc::new(JsonConfig::new()); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let exec = DataSourceExec::new(source); + let exec = FileSourceConfig::new_exec(conf, source_config); // TODO: this is not where schema inference should be tested @@ -463,8 +461,7 @@ mod tests { .with_file_compression_type(file_compression_type.to_owned()); let source_config = Arc::new(JsonConfig::new()); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let exec = DataSourceExec::new(source); + let exec = FileSourceConfig::new_exec(conf, source_config); let mut it = exec.execute(0, task_ctx)?; let batch = it.next().await.unwrap()?; @@ -505,8 +502,7 @@ mod tests { .with_file_compression_type(file_compression_type.to_owned()); let source_config = Arc::new(JsonConfig::new()); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let exec = DataSourceExec::new(source); + let exec = FileSourceConfig::new_exec(conf, source_config); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 2); @@ -552,8 +548,7 @@ mod tests { .with_file_compression_type(file_compression_type.to_owned()); let source_config = Arc::new(JsonConfig::new()); - let source = Arc::new(FileSourceConfig::new(conf, source_config)); - let exec = DataSourceExec::new(source); + let exec = FileSourceConfig::new_exec(conf, source_config); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 3); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 5065ee87fdd2..47fd5a74ded4 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -112,7 +112,7 @@ pub use writer::plan_to_parquet; /// TableParquetOptions::default() /// ) /// ); -/// let exec = DataSourceExec::new(Arc::new(FileSourceConfig::new(file_scan_config, source_config))); +/// let exec = FileSourceConfig::new_exec(file_scan_config, source_config); /// ``` /// /// # Features @@ -196,7 +196,7 @@ pub use writer::plan_to_parquet; /// .clone() /// .with_file_groups(vec![file_group.clone()]); /// -/// DataSourceExec::new(Arc::new(FileSourceConfig::new(new_config, source_config.clone()))) +/// FileSourceConfig::new_exec(new_config, source_config.clone()) /// }) /// .collect::>(); /// ``` @@ -241,7 +241,7 @@ pub use writer::plan_to_parquet; /// let source_config = Arc::new(ParquetConfig::default()); /// // this parquet exec will not even try to read row groups 2 and 4. Additional /// // pruning based on predicates may also happen -/// let exec = DataSourceExec::new(Arc::new(FileSourceConfig::new(file_scan_config, source_config))); +/// let exec = FileSourceConfig::new_exec(file_scan_config, source_config); /// ``` /// /// For a complete example, see the [`advanced_parquet_index` example]). @@ -763,11 +763,8 @@ mod tests { let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); - let source = Arc::new(FileSourceConfig::new( - base_config, - Arc::new(source_config.clone()), - )); - let parquet_exec = Arc::new(DataSourceExec::new(source)); + let parquet_exec = + FileSourceConfig::new_exec(base_config, Arc::new(source_config.clone())); RoundTripResult { batches: collect(parquet_exec.clone(), task_ctx).await, parquet_exec, @@ -1402,12 +1399,11 @@ mod tests { expected_row_num: Option, file_schema: SchemaRef, ) -> Result<()> { - let source = Arc::new(FileSourceConfig::new( + let parquet_exec = FileSourceConfig::new_exec( FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file_groups(file_groups), Arc::new(ParquetConfig::default()), - )); - let parquet_exec = DataSourceExec::new(source); + ); assert_eq!( parquet_exec .properties() @@ -1505,7 +1501,7 @@ mod tests { ]); let source_config = Arc::new(ParquetConfig::default()); - let parquet_exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( + let parquet_exec = FileSourceConfig::new_exec( 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 @@ -1523,7 +1519,7 @@ mod tests { ), ]), source_config, - ))); + ); let partition_count = parquet_exec .source() .properties() @@ -1581,11 +1577,11 @@ mod tests { }; let file_schema = Arc::new(Schema::empty()); - let parquet_exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( + let parquet_exec = FileSourceConfig::new_exec( FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file(partitioned_file), Arc::new(ParquetConfig::default()), - ))); + ); let mut results = parquet_exec.execute(0, state.task_ctx())?; let batch = results.next().await.unwrap(); @@ -2210,7 +2206,7 @@ mod tests { .with_parquet_file_reader_factory(reader_factory) .with_metadata_size_hint(456), ); - let exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( + let exec = FileSourceConfig::new_exec( FileScanConfig::new(store_url, schema) .with_file( PartitionedFile { @@ -2244,9 +2240,8 @@ mod tests { metadata_size_hint: None, }), source_config, - ))); + ); - 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/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index a5bc66ac3318..64b3d230f66e 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -452,7 +452,6 @@ mod tests { use crate::prelude::SessionContext; use datafusion_common::record_batch; - use datafusion_physical_plan::source::DataSourceExec; #[cfg(feature = "parquet")] use parquet::arrow::ArrowWriter; use tempfile::TempDir; @@ -510,12 +509,11 @@ mod tests { .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})), ); - let source = Arc::new(FileSourceConfig::new(base_conf, source_config)); - let parquet_exec = DataSourceExec::new(source); + let parquet_exec = FileSourceConfig::new_exec(base_conf, source_config); 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/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 5d98734cdb5d..845e43dad0f6 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -243,12 +243,12 @@ fn try_swapping_with_memory( memory.projection().as_ref().unwrap_or(&all_projections), ); - let source = Arc::new(MemorySourceConfig::try_new( + let exec = MemorySourceConfig::try_new_exec( memory.partitions(), memory.original_schema(), Some(new_projections), - )?); - Ok(Arc::new(DataSourceExec::new(source)) as _) + )? as _; + Ok(exec) }) .transpose() } @@ -1788,10 +1788,7 @@ mod tests { Field::new("e", DataType::Int32, true), ])); - let source = Arc::new( - MemorySourceConfig::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap(), - ); - Arc::new(DataSourceExec::new(source)) + MemorySourceConfig::try_new_exec(&[], schema, Some(vec![2, 0, 3, 4])).unwrap() } #[test] diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 305fa9481088..d34106cdcad1 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -218,9 +218,7 @@ pub fn coalesce_partitions_exec(input: Arc) -> Arc Arc { - let source = - Arc::new(MemorySourceConfig::try_new(&[vec![]], schema.clone(), None).unwrap()); - Arc::new(DataSourceExec::new(source)) + MemorySourceConfig::try_new_exec(&[vec![]], schema.clone(), None).unwrap() } pub fn hash_join_exec( diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index c89b4b2aa5da..25b1de78efb4 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -84,7 +84,6 @@ use datafusion_physical_expr::LexOrdering; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::memory::MemorySourceConfig; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; -use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::unnest::ListUnnest; use datafusion_sql::utils::window_expr_common_partition_keys; @@ -1941,12 +1940,8 @@ impl DefaultPhysicalPlanner { let schema = record_batch.schema(); let partitions = vec![vec![record_batch]]; let projection = None; - let mem_exec = DataSourceExec::new(Arc::new(MemorySourceConfig::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/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index b32794f7a4b5..a89502245816 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -304,11 +304,12 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str let concat_input_record = concat_batches(&schema, &input1).unwrap(); - let source = Arc::new( - MemorySourceConfig::try_new(&[vec![concat_input_record]], schema.clone(), None) - .unwrap(), - ); - let usual_source = Arc::new(DataSourceExec::new(source)); + 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) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 17bf18b8dccf..6ea80ad154dd 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -428,16 +428,13 @@ impl JoinFuzzTestCase { fn left_right(&self) -> (Arc, Arc) { let schema1 = self.input1[0].schema(); let schema2 = self.input2[0].schema(); - let left = Arc::new( - MemorySourceConfig::try_new(&[self.input1.clone()], schema1, None).unwrap(), - ); - let right = Arc::new( - MemorySourceConfig::try_new(&[self.input2.clone()], schema2, None).unwrap(), - ); - ( - Arc::new(DataSourceExec::new(left)), - Arc::new(DataSourceExec::new(right)), - ) + let left = + MemorySourceConfig::try_new_exec(&[self.input1.clone()], schema1, None) + .unwrap(); + let right = + MemorySourceConfig::try_new_exec(&[self.input2.clone()], schema2, None) + .unwrap(); + (left, right) } fn join_filter(&self) -> Option { diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index f828b51178c3..95404fa32ea8 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -32,7 +32,6 @@ use datafusion::physical_plan::{ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; #[tokio::test] @@ -117,9 +116,8 @@ async fn run_merge_test(input: Vec>) { }, }]); - let config = MemorySourceConfig::try_new(&input, schema, None).unwrap(); - let exec = DataSourceExec::new(Arc::new(config)); - 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 d9617e2b654d..08157207bf3f 100644 --- a/datafusion/core/tests/fuzz_cases/pruning.rs +++ b/datafusion/core/tests/fuzz_cases/pruning.rs @@ -30,7 +30,6 @@ use datafusion_common::config::TableParquetOptions; use datafusion_common::DFSchema; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{collect, filter::FilterExec, ExecutionPlan}; use itertools::Itertools; use object_store::{memory::InMemory, path::Path, ObjectStore, PutPayload}; @@ -332,8 +331,7 @@ async fn execute_with_predicate( TableParquetOptions::default(), ) }; - let source = Arc::new(FileSourceConfig::new(scan, Arc::new(parquet_conf))); - let exec = Arc::new(DataSourceExec::new(source)) as Arc; + let exec = FileSourceConfig::new_exec(scan, Arc::new(parquet_conf)); 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 6403eb0f71ed..8a989a41e9e1 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -31,7 +31,6 @@ 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 datafusion_physical_plan::source::DataSourceExec; use rand::Rng; use std::sync::Arc; use test_utils::{batches_to_vec, partitions_to_sorted_vec}; @@ -124,9 +123,8 @@ impl SortTest { }, }]); - let config = MemorySourceConfig::try_new(&input, schema, None).unwrap(); - let exec = DataSourceExec::new(Arc::new(config)); - 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/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index fdd9a62e7ab8..e1517b556557 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -160,12 +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_conf = Arc::new(MemorySourceConfig::try_new( - &[batches.clone()], - schema.clone(), - None, - )?); - let memory_exec = Arc::new(DataSourceExec::new(memory_conf)); + let memory_exec = + MemorySourceConfig::try_new_exec(&[batches.clone()], schema.clone(), None)?; // Different window functions to test causality let window_functions = vec![ diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 1cb273ea5af2..85c0374a7a3f 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -35,7 +35,6 @@ use datafusion::physical_plan::collect; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::prelude::SessionContext; use datafusion_common::Result; -use datafusion_physical_plan::source::DataSourceExec; use bytes::Bytes; use futures::future::BoxFuture; @@ -91,12 +90,11 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { )), ); - let parquet_exec = - DataSourceExec::new(Arc::new(FileSourceConfig::new(base_config, source_config))); + let parquet_exec = FileSourceConfig::new_exec(base_config, source_config); 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/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index a47ad800fa5c..4f08bcd2267a 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -29,7 +29,6 @@ use datafusion::physical_plan::collect; use datafusion::prelude::SessionContext; use datafusion_common::Result; use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_physical_plan::source::DataSourceExec; use object_store::path::Path; use object_store::ObjectMeta; @@ -63,15 +62,15 @@ async fn multi_parquet_coercion() { ])); let source_config = Arc::new(ParquetConfig::default()); - let parquet_exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( + let parquet_exec = FileSourceConfig::new_exec( FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file_group(file_group), source_config, - ))); + ); 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 = [ "+-------+----+------+", @@ -117,16 +116,16 @@ async fn multi_parquet_coercion_projection() { Field::new("c2", DataType::Int32, true), Field::new("c3", DataType::Float64, true), ])); - let parquet_exec = DataSourceExec::new(Arc::new(FileSourceConfig::new( + let parquet_exec = FileSourceConfig::new_exec( FileScanConfig::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file_group(file_group) .with_projection(Some(vec![1, 0, 2])), Arc::new(ParquetConfig::default()), - ))); + ); 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/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 0d19120c1711..9acf2e521f7c 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1019,7 +1019,7 @@ mod tests { ErrorExec, MockExec, }, }, - {collect, expressions::col, memory::MemorySourceConfig, source::DataSourceExec}, + {collect, expressions::col, memory::MemorySourceConfig}, }; use arrow::array::{ArrayRef, StringArray, UInt32Array}; @@ -1124,12 +1124,12 @@ mod tests { ) -> Result>> { let task_ctx = Arc::new(TaskContext::default()); // create physical plan - let exec = DataSourceExec::new(Arc::new(MemorySourceConfig::try_new( + let exec = MemorySourceConfig::try_new_exec( &input_partitions, Arc::clone(schema), None, - )?)); - let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?; + )?; + let exec = RepartitionExec::try_new(exec, partitioning)?; // execute and collect results let mut output_partitions = vec![]; @@ -1519,12 +1519,12 @@ mod tests { let task_ctx = Arc::new(task_ctx); // create physical plan - let exec = DataSourceExec::new(Arc::new(MemorySourceConfig::try_new( + let exec = MemorySourceConfig::try_new_exec( &input_partitions, Arc::clone(&schema), None, - )?)); - let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?; + )?; + let exec = RepartitionExec::try_new(exec, partitioning)?; // pull partitions for i in 0..exec.partitioning().partition_count() { diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index 84974cb787ec..6e1e4936b3e1 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); diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index c6afbed3319c..8204f1c0594c 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -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 diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index c308fbb9d645..d1e8c95964c1 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 From ff68caaf5936a20d9e856015542113627bdb29d7 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 15 Jan 2025 16:00:59 +0300 Subject: [PATCH 16/39] fix imports and comment line --- datafusion/core/src/datasource/physical_plan/csv.rs | 4 ++-- .../src/datasource/physical_plan/file_scan_config.rs | 2 +- datafusion/physical-plan/src/source.rs | 10 ++++++---- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index b46f2d8684c1..1e5706d89ee7 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -412,6 +412,7 @@ mod tests { use super::*; use crate::dataframe::DataFrameWriteOptions; + use crate::datasource::data_source::FileSourceConfig; use crate::datasource::file_format::csv::CsvFormat; use crate::prelude::*; use crate::test::{partitioned_csv_config, partitioned_file_groups}; @@ -420,9 +421,8 @@ mod tests { use arrow::datatypes::*; use bytes::Bytes; use datafusion_common::test_util::arrow_test_data; - - use crate::datasource::data_source::FileSourceConfig; use datafusion_physical_plan::metrics::MetricsSet; + use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; 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 540d5280fd57..3df5f000b819 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -130,7 +130,7 @@ pub struct FileScanConfig { pub output_ordering: Vec, /// File compression type pub file_compression_type: FileCompressionType, - /// New Lines in Values for CSVOptions + /// Are new lines in values supported for CSVOptions pub new_lines_in_values: bool, } diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs index ebfc17321b22..60ebe68c4d62 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/physical-plan/src/source.rs @@ -15,15 +15,17 @@ // 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::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; + use datafusion_common::config::ConfigOptions; use datafusion_common::Statistics; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use std::any::Any; -use std::fmt; -use std::fmt::{Debug, Formatter}; -use std::sync::Arc; /// Common behaviors in Data Sources for both from Files and Memory. /// See `DataSourceExec` for physical plan implementation From 7a62190245388726608711df3972dea988307fa6 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 15 Jan 2025 16:22:08 +0300 Subject: [PATCH 17/39] simplify some lines --- datafusion-cli/src/functions.rs | 5 +- .../examples/advanced_parquet_index.rs | 3 +- .../examples/csv_json_opener.rs | 1 + datafusion-examples/examples/parquet_index.rs | 3 +- datafusion-examples/examples/simple_udtf.rs | 5 +- datafusion/core/benches/physical_plan.rs | 2 +- datafusion/core/benches/sort.rs | 3 +- datafusion/core/src/datasource/data_source.rs | 63 +++++++------------ .../core/src/datasource/file_format/arrow.rs | 4 +- .../core/src/datasource/file_format/avro.rs | 7 ++- .../core/src/datasource/file_format/csv.rs | 3 +- .../core/src/datasource/file_format/json.rs | 3 +- .../src/datasource/file_format/parquet.rs | 3 +- .../core/src/datasource/physical_plan/csv.rs | 2 - .../core/src/datasource/physical_plan/json.rs | 5 -- .../physical_optimizer/projection_pushdown.rs | 5 +- 16 files changed, 40 insertions(+), 77 deletions(-) diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index 212ddada4ace..4f8271eb8f59 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -243,12 +243,11 @@ impl TableProvider for ParquetMetadataTable { _filters: &[Expr], _limit: Option, ) -> Result> { - let exec = MemorySourceConfig::try_new_exec( + Ok(MemorySourceConfig::try_new_exec( &[vec![self.batch.clone()]], TableProvider::schema(self), projection.cloned(), - )?; - Ok(exec) + )?) } } diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 9cd50138f3b2..6a5c36778da4 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -509,8 +509,7 @@ impl TableProvider for IndexTableProvider { .with_parquet_file_reader_factory(Arc::new(reader_factory)), ); // Finally, put it all together into a DataSourceExec - let exec = FileSourceConfig::new_exec(file_scan_config, source_config); - Ok(exec) + Ok(FileSourceConfig::new_exec(file_scan_config, source_config)) } /// 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 9f15c710381a..9b2e75b82e50 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -31,6 +31,7 @@ use datafusion::{ physical_plan::metrics::ExecutionPlanMetricsSet, test_util::aggr_test_schema, }; + use futures::StreamExt; use object_store::{local::LocalFileSystem, memory::InMemory, ObjectStore}; diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index 1d11df49798f..1cc30fd21818 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -263,8 +263,7 @@ impl TableProvider for IndexTableProvider { None, TableParquetOptions::default(), )); - let exec = FileSourceConfig::new_exec(file_scan_config, source_config); - Ok(exec) + Ok(FileSourceConfig::new_exec(file_scan_config, source_config)) } /// Tell DataFusion to push filters down to the scan method diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index bc6ff32e6655..e6250a7814be 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -120,12 +120,11 @@ impl TableProvider for LocalCsvTable { } else { self.batches.clone() }; - let exec = MemorySourceConfig::try_new_exec( + Ok(MemorySourceConfig::try_new_exec( &[batches], TableProvider::schema(self), projection.cloned(), - )?; - Ok(exec) + )?) } } diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index a75de235d1dc..53c245ecc2b5 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -33,10 +33,10 @@ use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMerge use datafusion::physical_plan::{ collect, expressions::{col, PhysicalSortExpr}, + memory::MemorySourceConfig, }; use datafusion::prelude::SessionContext; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::memory::MemorySourceConfig; // Initialize the operator using the provided record batches and the sort key // as inputs. All record batches must have the same schema. diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 0e2280c8795b..4d71d4c56a6d 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -79,7 +79,7 @@ use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::{ execution::context::TaskContext, physical_plan::{ - coalesce_partitions::CoalescePartitionsExec, + coalesce_partitions::CoalescePartitionsExec, memory::MemorySourceConfig, sorts::sort_preserving_merge::SortPreservingMergeExec, ExecutionPlan, ExecutionPlanProperties, }, @@ -87,7 +87,6 @@ use datafusion::{ }; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::memory::MemorySourceConfig; /// Benchmarks for SortPreservingMerge stream use criterion::{criterion_group, criterion_main, Criterion}; diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 4f259d992db1..815e8cc59a08 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! DataSource trait implementation +//! DataSource and DataSourceFileConfig trait implementations use std::any::Any; use std::fmt; @@ -246,7 +246,6 @@ impl FileSourceConfig { source_config: Arc, ) -> Arc { let source = Arc::new(Self::new(base_config, source_config)); - Arc::new(DataSourceExec::new(source)) } @@ -284,46 +283,26 @@ impl FileSourceConfig { /// Write the data_type based on source_config fn fmt_source_config(&self, f: &mut Formatter) -> fmt::Result { - let mut data_type = if self - .source_config - .as_any() - .downcast_ref::() - .is_some() - { - "avro" - } else if self - .source_config - .as_any() - .downcast_ref::() - .is_some() - { - "arrow" - } else if self - .source_config - .as_any() - .downcast_ref::() - .is_some() - { - "csv" - } else if self - .source_config - .as_any() - .downcast_ref::() - .is_some() - { - "json" - } else { - "unknown" - }; - #[cfg(feature = "parquet")] - if self - .source_config - .as_any() - .downcast_ref::() - .is_some() - { - data_type = "parquet"; - } + let source_config = self.source_config.as_any(); + let data_type = [ + ("avro", source_config.downcast_ref::().is_some()), + ( + "arrow", + source_config.downcast_ref::().is_some(), + ), + ("csv", source_config.downcast_ref::().is_some()), + ("json", source_config.downcast_ref::().is_some()), + #[cfg(feature = "parquet")] + ( + "parquet", + source_config.downcast_ref::().is_some(), + ), + ] + .iter() + .find(|(_, is_some)| *is_some) + .map(|(name, _)| *name) + .unwrap_or("unknown"); + write!(f, ", file_type={}", data_type) } diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 634a9116b517..6fb086ee2611 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -173,9 +173,7 @@ impl FileFormat for ArrowFormat { conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { - let source_config = Arc::new(ArrowConfig {}); - let exec = FileSourceConfig::new_exec(conf, source_config); - Ok(exec) + Ok(FileSourceConfig::new_exec(conf, Arc::new(ArrowConfig {}))) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index bde69da39f98..8d9b6121e926 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -151,9 +151,10 @@ impl FileFormat for AvroFormat { conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { - let source_config = Arc::new(AvroConfig::new()); - let exec = FileSourceConfig::new_exec(conf, source_config); - Ok(exec) + Ok(FileSourceConfig::new_exec( + conf, + Arc::new(AvroConfig::new()), + )) } } diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 8508b213f90f..99144b16c725 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -433,8 +433,7 @@ impl FileFormat for CsvFormat { .with_comment(self.options.comment), ); - let exec = FileSourceConfig::new_exec(conf, source_config); - Ok(exec) + Ok(FileSourceConfig::new_exec(conf, source_config)) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 893d9dd8eede..38cdd657276d 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -254,8 +254,7 @@ impl FileFormat for JsonFormat { let source_config = Arc::new(JsonConfig::new()); conf.file_compression_type = FileCompressionType::from(self.options.compression); - let exec = FileSourceConfig::new_exec(conf, source_config); - Ok(exec) + Ok(FileSourceConfig::new_exec(conf, source_config)) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 92213bf31bed..fc907192c861 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -423,8 +423,7 @@ impl FileFormat for ParquetFormat { metadata_size_hint, self.options.clone(), )); - let exec = FileSourceConfig::new_exec(conf, source_config); - Ok(exec) + Ok(FileSourceConfig::new_exec(conf, source_config)) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 1e5706d89ee7..ee615267ae85 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -529,7 +529,6 @@ mod tests { .with_newlines_in_values(false) .with_file_compression_type(file_compression_type.to_owned()); config.projection = Some(vec![4, 0, 2]); - let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); let csv = FileSourceConfig::new_exec(config.clone(), source_config); assert_eq!(13, config.file_schema.fields().len()); @@ -593,7 +592,6 @@ mod tests { .with_newlines_in_values(false) .with_file_compression_type(file_compression_type.to_owned()); config.limit = Some(5); - let source_config = Arc::new(CsvConfig::new(true, b',', b'"')); let csv = FileSourceConfig::new_exec(config.clone(), source_config); assert_eq!(13, config.file_schema.fields().len()); diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 349eceadddc0..d586068562d4 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -386,9 +386,7 @@ mod tests { .with_file_groups(file_groups) .with_limit(Some(3)) .with_file_compression_type(file_compression_type.to_owned()); - let source_config = Arc::new(JsonConfig::new()); - let exec = FileSourceConfig::new_exec(conf, source_config); // TODO: this is not where schema inference should be tested @@ -459,7 +457,6 @@ mod tests { .with_file_groups(file_groups) .with_limit(Some(3)) .with_file_compression_type(file_compression_type.to_owned()); - let source_config = Arc::new(JsonConfig::new()); let exec = FileSourceConfig::new_exec(conf, source_config); @@ -500,7 +497,6 @@ mod tests { .with_file_groups(file_groups) .with_projection(Some(vec![0, 2])) .with_file_compression_type(file_compression_type.to_owned()); - let source_config = Arc::new(JsonConfig::new()); let exec = FileSourceConfig::new_exec(conf, source_config); let inferred_schema = exec.schema(); @@ -546,7 +542,6 @@ mod tests { .with_file_groups(file_groups) .with_projection(Some(vec![3, 0, 2])) .with_file_compression_type(file_compression_type.to_owned()); - let source_config = Arc::new(JsonConfig::new()); let exec = FileSourceConfig::new_exec(conf, source_config); let inferred_schema = exec.schema(); diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 845e43dad0f6..7df5aacab7c9 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -243,12 +243,11 @@ fn try_swapping_with_memory( memory.projection().as_ref().unwrap_or(&all_projections), ); - let exec = MemorySourceConfig::try_new_exec( + Ok(MemorySourceConfig::try_new_exec( memory.partitions(), memory.original_schema(), Some(new_projections), - )? as _; - Ok(exec) + )? as _) }) .transpose() } From fd3718304b4b3e4acd8a38d7b4159bdf2011a773 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 15 Jan 2025 17:33:35 +0300 Subject: [PATCH 18/39] rename source_config as file_source --- .../examples/csv_json_opener.rs | 2 +- .../examples/parquet_exec_visitor.rs | 2 +- datafusion/core/src/datasource/data_source.rs | 79 ++++++++----------- .../datasource/physical_plan/arrow_file.rs | 10 +-- .../core/src/datasource/physical_plan/avro.rs | 10 +-- .../core/src/datasource/physical_plan/csv.rs | 10 +-- .../core/src/datasource/physical_plan/json.rs | 10 +-- .../datasource/physical_plan/parquet/mod.rs | 14 ++-- .../physical_optimizer/projection_pushdown.rs | 4 +- datafusion/core/src/test_util/parquet.rs | 2 +- datafusion/core/tests/parquet/utils.rs | 2 +- datafusion/core/tests/sql/path_partition.rs | 2 +- datafusion/proto/src/physical_plan/mod.rs | 9 +-- .../substrait/src/physical_plan/producer.rs | 2 +- 14 files changed, 69 insertions(+), 89 deletions(-) diff --git a/datafusion-examples/examples/csv_json_opener.rs b/datafusion-examples/examples/csv_json_opener.rs index 9b2e75b82e50..1acc80c4e2db 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use arrow_schema::{DataType, Field, Schema}; -use datafusion::datasource::data_source::DataSourceFileConfig; +use datafusion::datasource::data_source::FileSource; use datafusion::{ assert_batches_eq, datasource::{ diff --git a/datafusion-examples/examples/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index a39cb0d6418b..fe281d1531a8 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -102,7 +102,7 @@ impl ExecutionPlanVisitor for ParquetExecVisitor { if let Some(file_config) = source.as_any().downcast_ref::() { if file_config - .source_config() + .file_source() .as_any() .downcast_ref::() .is_some() diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 815e8cc59a08..f5e3e6b80b92 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -47,8 +47,7 @@ use itertools::Itertools; use object_store::ObjectStore; /// Common behaviors that every `FileSourceConfig` needs to implement. -/// Being stored as source_config in `DataSourceExec`. -pub trait DataSourceFileConfig: Send + Sync { +pub trait FileSource: Send + Sync { /// Creates a `dyn FileOpener` based on given parameters fn create_file_opener( &self, @@ -56,16 +55,14 @@ pub trait DataSourceFileConfig: Send + Sync { 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; + fn with_batch_size(&self, batch_size: usize) -> Arc; /// Initialize new instance with a new schema - fn with_schema(&self, schema: SchemaRef) -> Arc; + fn with_schema(&self, schema: SchemaRef) -> Arc; /// Initialize new instance with projection information - fn with_projection(&self, config: &FileScanConfig) -> Arc; + fn with_projection(&self, config: &FileScanConfig) -> Arc; } /// Holds generic file configuration, and common behaviors. @@ -73,7 +70,7 @@ pub trait DataSourceFileConfig: Send + Sync { /// and a `dyn DataSourceFileConfig` type such as `CsvConfig`, `ParquetConfig`, `AvroConfig`, etc. #[derive(Clone)] pub struct FileSourceConfig { - source_config: Arc, + source: Arc, base_config: FileScanConfig, metrics: ExecutionPlanMetricsSet, projected_statistics: Statistics, @@ -90,17 +87,14 @@ impl DataSource for FileSourceConfig { .runtime_env() .object_store(&self.base_config.object_store_url); - let source_config = self - .source_config + let source = self + .source .with_batch_size(context.session_config().batch_size()) .with_schema(Arc::clone(&self.base_config.file_schema)) .with_projection(&self.base_config); - let opener = source_config.create_file_opener( - object_store, - &self.base_config, - partition, - )?; + let opener = + source.create_file_opener(object_store, &self.base_config, partition)?; let stream = FileStream::new(&self.base_config, partition, opener, &self.metrics)?; @@ -113,16 +107,16 @@ impl DataSource for FileSourceConfig { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { self.base_config.fmt_as(t, f)?; - self.fmt_source_config(f)?; + self.fmt_file_source(f)?; - if let Some(csv_conf) = self.source_config.as_any().downcast_ref::() { + if let Some(csv_conf) = self.source.as_any().downcast_ref::() { return write!(f, ", has_header={}", csv_conf.has_header); } #[cfg(feature = "parquet")] if cfg!(feature = "parquet") { if let Some(parquet_conf) = - self.source_config.as_any().downcast_ref::() + self.source.as_any().downcast_ref::() { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { @@ -194,7 +188,7 @@ impl DataSource for FileSourceConfig { #[cfg(feature = "parquet")] let stats = if let Some(parquet_config) = - self.source_config.as_any().downcast_ref::() + self.source.as_any().downcast_ref::() { // 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. @@ -218,7 +212,7 @@ impl DataSource for FileSourceConfig { fn with_fetch(&self, limit: Option) -> Option> { let config = self.base_config.clone().with_limit(limit); Some(Arc::new(Self { - source_config: Arc::clone(&self.source_config), + source: Arc::clone(&self.source), base_config: config, metrics: self.metrics.clone(), projected_statistics: self.projected_statistics.clone(), @@ -243,17 +237,14 @@ impl FileSourceConfig { /// Returns a new [`DataSourceExec`] from file configurations pub fn new_exec( base_config: FileScanConfig, - source_config: Arc, + file_source: Arc, ) -> Arc { - let source = Arc::new(Self::new(base_config, source_config)); + let source = Arc::new(Self::new(base_config, file_source)); Arc::new(DataSourceExec::new(source)) } /// Initialize a new `FileSourceConfig` instance with metrics, cache, and statistics. - pub fn new( - base_config: FileScanConfig, - source_config: Arc, - ) -> Self { + pub fn new(base_config: FileScanConfig, file_source: Arc) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); let cache = Self::compute_properties( @@ -264,8 +255,7 @@ impl FileSourceConfig { let mut metrics = ExecutionPlanMetricsSet::new(); #[cfg(feature = "parquet")] - if let Some(parquet_config) = - source_config.as_any().downcast_ref::() + if let Some(parquet_config) = file_source.as_any().downcast_ref::() { metrics = parquet_config.metrics(); let _predicate_creation_errors = MetricBuilder::new(&metrics) @@ -273,7 +263,7 @@ impl FileSourceConfig { }; Self { - source_config, + source: file_source, base_config, metrics, projected_statistics, @@ -281,21 +271,18 @@ impl FileSourceConfig { } } - /// Write the data_type based on source_config - fn fmt_source_config(&self, f: &mut Formatter) -> fmt::Result { - let source_config = self.source_config.as_any(); + /// Write the data_type based on file_source + fn fmt_file_source(&self, f: &mut Formatter) -> fmt::Result { + let file_source = self.source.as_any(); let data_type = [ - ("avro", source_config.downcast_ref::().is_some()), - ( - "arrow", - source_config.downcast_ref::().is_some(), - ), - ("csv", source_config.downcast_ref::().is_some()), - ("json", source_config.downcast_ref::().is_some()), + ("avro", file_source.downcast_ref::().is_some()), + ("arrow", file_source.downcast_ref::().is_some()), + ("csv", file_source.downcast_ref::().is_some()), + ("json", file_source.downcast_ref::().is_some()), #[cfg(feature = "parquet")] ( "parquet", - source_config.downcast_ref::().is_some(), + file_source.downcast_ref::().is_some(), ), ] .iter() @@ -311,9 +298,9 @@ impl FileSourceConfig { &self.base_config } - /// Returns the source_config - pub fn source_config(&self) -> &Arc { - &self.source_config + /// Returns the file_source + pub fn file_source(&self) -> &Arc { + &self.source } /// Returns the `PlanProperties` of the plan @@ -352,10 +339,6 @@ impl FileSourceConfig { fn supports_repartition(&self) -> bool { !(self.base_config.file_compression_type.is_compressed() || self.base_config.new_lines_in_values - || self - .source_config - .as_any() - .downcast_ref::() - .is_some()) + || self.source.as_any().downcast_ref::().is_some()) } } diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 81fe134c5e14..c4822c61a447 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use crate::datasource::data_source::DataSourceFileConfig; +use crate::datasource::data_source::FileSource; use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, FileScanConfig, }; @@ -38,7 +38,7 @@ use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; #[derive(Clone, Default)] pub struct ArrowConfig {} -impl DataSourceFileConfig for ArrowConfig { +impl FileSource for ArrowConfig { fn create_file_opener( &self, object_store: Result>, @@ -55,15 +55,15 @@ impl DataSourceFileConfig for ArrowConfig { self } - fn with_batch_size(&self, _batch_size: usize) -> Arc { + fn with_batch_size(&self, _batch_size: usize) -> Arc { Arc::new(ArrowConfig::default()) } - fn with_schema(&self, _schema: SchemaRef) -> Arc { + fn with_schema(&self, _schema: SchemaRef) -> Arc { Arc::new(ArrowConfig::default()) } - fn with_projection(&self, _config: &FileScanConfig) -> Arc { + fn with_projection(&self, _config: &FileScanConfig) -> Arc { Arc::new(ArrowConfig::default()) } } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 0d3daa0ac491..d824f0fc02a5 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use super::{FileOpener, FileScanConfig}; #[cfg(feature = "avro")] use crate::datasource::avro_to_arrow::Reader as AvroReader; -use crate::datasource::data_source::DataSourceFileConfig; +use crate::datasource::data_source::FileSource; use crate::error::Result; use arrow::datatypes::SchemaRef; @@ -55,7 +55,7 @@ impl AvroConfig { } } -impl DataSourceFileConfig for AvroConfig { +impl FileSource for AvroConfig { #[cfg(feature = "avro")] fn create_file_opener( &self, @@ -83,19 +83,19 @@ impl DataSourceFileConfig for AvroConfig { self } - fn with_batch_size(&self, batch_size: usize) -> Arc { + 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 { + fn with_schema(&self, schema: SchemaRef) -> Arc { let mut conf = self.clone(); conf.schema = Some(schema); Arc::new(conf) } - fn with_projection(&self, config: &FileScanConfig) -> Arc { + fn with_projection(&self, config: &FileScanConfig) -> Arc { let mut conf = self.clone(); conf.projection = config.projected_file_column_names(); Arc::new(conf) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index ee615267ae85..545504f34800 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -23,6 +23,7 @@ use std::sync::Arc; use std::task::Poll; 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}; @@ -35,7 +36,6 @@ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use crate::datasource::data_source::DataSourceFileConfig; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; use object_store::{GetOptions, GetResultPayload, ObjectStore}; @@ -210,7 +210,7 @@ impl CsvOpener { } } -impl DataSourceFileConfig for CsvConfig { +impl FileSource for CsvConfig { fn create_file_opener( &self, object_store: Result>, @@ -228,19 +228,19 @@ impl DataSourceFileConfig for CsvConfig { self } - fn with_batch_size(&self, batch_size: usize) -> Arc { + 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 { + fn with_schema(&self, schema: SchemaRef) -> Arc { let mut conf = self.clone(); conf.file_schema = Some(schema); Arc::new(conf) } - fn with_projection(&self, config: &FileScanConfig) -> Arc { + fn with_projection(&self, config: &FileScanConfig) -> Arc { let mut conf = self.clone(); conf.file_projection = config.file_column_projection_indices(); Arc::new(conf) diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index d586068562d4..280bdfc88c38 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -30,12 +30,12 @@ use crate::datasource::physical_plan::file_stream::{FileOpenFuture, FileOpener}; use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; +use crate::datasource::data_source::FileSource; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_execution::TaskContext; -use crate::datasource::data_source::DataSourceFileConfig; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; use object_store::{GetOptions, GetResultPayload, ObjectStore}; @@ -80,7 +80,7 @@ impl JsonConfig { } } -impl DataSourceFileConfig for JsonConfig { +impl FileSource for JsonConfig { fn create_file_opener( &self, object_store: Result>, @@ -101,17 +101,17 @@ impl DataSourceFileConfig for JsonConfig { self } - fn with_batch_size(&self, batch_size: usize) -> Arc { + 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 { + fn with_schema(&self, _schema: SchemaRef) -> Arc { Arc::new(Self { ..*self }) } - fn with_projection(&self, _config: &FileScanConfig) -> Arc { + fn with_projection(&self, _config: &FileScanConfig) -> Arc { Arc::new(Self { ..*self }) } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 47fd5a74ded4..4679cd972c8b 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use crate::datasource::data_source::DataSourceFileConfig; +use crate::datasource::data_source::FileSource; use crate::datasource::physical_plan::{ parquet::page_filter::PagePruningAccessPlanFilter, FileOpener, FileScanConfig, }; @@ -186,7 +186,7 @@ pub use writer::plan_to_parquet; /// let source = parquet_exec().source(); /// let data_source = source.as_any().downcast_ref::().unwrap(); /// let base_config = data_source.base_config(); -/// let source_config = data_source.source_config(); +/// let file_source = data_source.file_source(); /// let existing_file_groups = &base_config.file_groups; /// let new_execs = existing_file_groups /// .iter() @@ -196,7 +196,7 @@ pub use writer::plan_to_parquet; /// .clone() /// .with_file_groups(vec![file_group.clone()]); /// -/// FileSourceConfig::new_exec(new_config, source_config.clone()) +/// FileSourceConfig::new_exec(new_config, file_source.clone()) /// }) /// .collect::>(); /// ``` @@ -524,7 +524,7 @@ impl ParquetConfig { } } -impl DataSourceFileConfig for ParquetConfig { +impl FileSource for ParquetConfig { fn create_file_opener( &self, object_store: Result>, @@ -575,17 +575,17 @@ impl DataSourceFileConfig for ParquetConfig { self } - fn with_batch_size(&self, batch_size: usize) -> Arc { + 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 { + fn with_schema(&self, _schema: SchemaRef) -> Arc { Arc::new(Self { ..self.clone() }) } - fn with_projection(&self, _config: &FileScanConfig) -> Arc { + fn with_projection(&self, _config: &FileScanConfig) -> Arc { Arc::new(Self { ..self.clone() }) } } diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 7df5aacab7c9..a58d8ebe4a6b 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -115,7 +115,7 @@ pub fn remove_unnecessary_projections( .downcast_ref::() .and_then(|file_config| { file_config - .source_config() + .file_source() .as_any() .downcast_ref::() .map(|_| true) @@ -223,7 +223,7 @@ fn try_swapping_with_csv( .unwrap_or((0..csv.schema().fields().len()).collect()), ); file_scan.projection = Some(new_projections); - FileSourceConfig::new_exec(file_scan, Arc::clone(csv_config.source_config())) as _ + FileSourceConfig::new_exec(file_scan, Arc::clone(csv_config.file_source())) as _ }) } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 5501bb3cb892..31ad66c4a487 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -206,7 +206,7 @@ impl TestParquetFile { source.as_any().downcast_ref::() { if maybe_parquet - .source_config() + .file_source() .as_any() .downcast_ref::() .is_some() diff --git a/datafusion/core/tests/parquet/utils.rs b/datafusion/core/tests/parquet/utils.rs index cca2b7b3ae4c..c9db55635219 100644 --- a/datafusion/core/tests/parquet/utils.rs +++ b/datafusion/core/tests/parquet/utils.rs @@ -53,7 +53,7 @@ impl ExecutionPlanVisitor for MetricsFinder { if let Some(file_config) = source.as_any().downcast_ref::() { if file_config - .source_config() + .file_source() .as_any() .downcast_ref::() .is_some() diff --git a/datafusion/core/tests/sql/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs index 97c6a8ff0b34..cb57867576d0 100644 --- a/datafusion/core/tests/sql/path_partition.rs +++ b/datafusion/core/tests/sql/path_partition.rs @@ -90,7 +90,7 @@ async fn parquet_partition_pruning_filter() -> Result<()> { let source = data_source.source(); let file_source = source.as_any().downcast_ref::().unwrap(); let parquet_config = file_source - .source_config() + .file_source() .as_any() .downcast_ref::() .unwrap(); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 56770f3228cd..312146ff32e4 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1611,12 +1611,9 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { if let Some(exec) = plan.downcast_ref::() { let source = exec.source(); - println!("DataSourceExec!"); if let Some(maybe_csv) = source.as_any().downcast_ref::() { - let source = maybe_csv.source_config(); - println!("Source!"); + let source = maybe_csv.file_source(); if let Some(csv_config) = source.as_any().downcast_ref::() { - println!("CSV!"); return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::CsvScan( protobuf::CsvScanExecNode { @@ -1665,7 +1662,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { if let Some(maybe_parquet) = source.as_any().downcast_ref::() { - let source = maybe_parquet.source_config(); + let source = maybe_parquet.file_source(); if let Some(conf) = source.as_any().downcast_ref::() { let predicate = conf .predicate() @@ -1689,7 +1686,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { 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.source_config(); + let source = maybe_avro.file_source(); if source.as_any().downcast_ref::().is_some() { return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::AvroScan( diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 086847c33431..649ae21c58dd 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -53,7 +53,7 @@ pub fn to_substrait_rel( let source = data_source.source(); if let Some(file_config) = source.as_any().downcast_ref::() { let is_parquet = file_config - .source_config() + .file_source() .as_any() .downcast_ref::() .is_some(); From cf13028003db5f9729f67ce943feb99fa5a90cb0 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 15 Jan 2025 17:33:54 +0300 Subject: [PATCH 19/39] format code --- datafusion/core/src/datasource/physical_plan/json.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 280bdfc88c38..a12aa76c437b 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -23,6 +23,7 @@ use std::sync::Arc; use std::task::Poll; 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; @@ -30,7 +31,6 @@ use crate::datasource::physical_plan::file_stream::{FileOpenFuture, FileOpener}; use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; -use crate::datasource::data_source::FileSource; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; From 201d8a00a37056798690652f147c42ef7f9d738c Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 15 Jan 2025 17:38:23 +0300 Subject: [PATCH 20/39] format code --- datafusion/core/src/datasource/data_source.rs | 226 +++++++++--------- 1 file changed, 113 insertions(+), 113 deletions(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index f5e3e6b80b92..ad777ce05ec0 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! DataSource and DataSourceFileConfig trait implementations +//! DataSource and FileSource trait implementations use std::any::Any; use std::fmt; @@ -65,9 +65,9 @@ pub trait FileSource: Send + Sync { fn with_projection(&self, config: &FileScanConfig) -> Arc; } -/// Holds generic file configuration, and common behaviors. +/// Holds generic file configuration, and common behaviors for file sources. /// Can be initialized with a `FileScanConfig` -/// and a `dyn DataSourceFileConfig` type such as `CsvConfig`, `ParquetConfig`, `AvroConfig`, etc. +/// and a `dyn FileSource` type such as `CsvConfig`, `ParquetConfig`, `AvroConfig`, etc. #[derive(Clone)] pub struct FileSourceConfig { source: Arc, @@ -77,6 +77,116 @@ pub struct FileSourceConfig { cache: PlanProperties, } +impl FileSourceConfig { + /// Returns a new [`DataSourceExec`] from file configurations + pub fn new_exec( + base_config: FileScanConfig, + file_source: Arc, + ) -> Arc { + let source = Arc::new(Self::new(base_config, file_source)); + Arc::new(DataSourceExec::new(source)) + } + + /// Initialize a new `FileSourceConfig` instance with metrics, cache, and statistics. + pub fn new(base_config: FileScanConfig, file_source: Arc) -> Self { + let (projected_schema, projected_statistics, projected_output_ordering) = + base_config.project(); + let cache = Self::compute_properties( + Arc::clone(&projected_schema), + &projected_output_ordering, + &base_config, + ); + let mut metrics = ExecutionPlanMetricsSet::new(); + + #[cfg(feature = "parquet")] + if let Some(parquet_config) = file_source.as_any().downcast_ref::() + { + metrics = parquet_config.metrics(); + let _predicate_creation_errors = MetricBuilder::new(&metrics) + .global_counter("num_predicate_creation_errors"); + }; + + Self { + source: file_source, + base_config, + metrics, + projected_statistics, + cache, + } + } + + /// Write the data_type based on file_source + fn fmt_file_source(&self, f: &mut Formatter) -> fmt::Result { + let file_source = self.source.as_any(); + let data_type = [ + ("avro", file_source.downcast_ref::().is_some()), + ("arrow", file_source.downcast_ref::().is_some()), + ("csv", file_source.downcast_ref::().is_some()), + ("json", file_source.downcast_ref::().is_some()), + #[cfg(feature = "parquet")] + ( + "parquet", + file_source.downcast_ref::().is_some(), + ), + ] + .iter() + .find(|(_, is_some)| *is_some) + .map(|(name, _)| *name) + .unwrap_or("unknown"); + + write!(f, ", file_type={}", data_type) + } + + /// Returns the base_config + pub fn base_config(&self) -> &FileScanConfig { + &self.base_config + } + + /// Returns the file_source + pub fn file_source(&self) -> &Arc { + &self.source + } + + /// Returns the `PlanProperties` of the plan + pub(crate) fn cache(&self) -> PlanProperties { + self.cache.clone() + } + + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + 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 + } + + fn supports_repartition(&self) -> bool { + !(self.base_config.file_compression_type.is_compressed() + || self.base_config.new_lines_in_values + || self.source.as_any().downcast_ref::().is_some()) + } +} + impl DataSource for FileSourceConfig { fn open( &self, @@ -232,113 +342,3 @@ impl DataSource for FileSourceConfig { self.cache() } } - -impl FileSourceConfig { - /// Returns a new [`DataSourceExec`] from file configurations - pub fn new_exec( - base_config: FileScanConfig, - file_source: Arc, - ) -> Arc { - let source = Arc::new(Self::new(base_config, file_source)); - Arc::new(DataSourceExec::new(source)) - } - - /// Initialize a new `FileSourceConfig` instance with metrics, cache, and statistics. - pub fn new(base_config: FileScanConfig, file_source: Arc) -> Self { - let (projected_schema, projected_statistics, projected_output_ordering) = - base_config.project(); - let cache = Self::compute_properties( - Arc::clone(&projected_schema), - &projected_output_ordering, - &base_config, - ); - let mut metrics = ExecutionPlanMetricsSet::new(); - - #[cfg(feature = "parquet")] - if let Some(parquet_config) = file_source.as_any().downcast_ref::() - { - metrics = parquet_config.metrics(); - let _predicate_creation_errors = MetricBuilder::new(&metrics) - .global_counter("num_predicate_creation_errors"); - }; - - Self { - source: file_source, - base_config, - metrics, - projected_statistics, - cache, - } - } - - /// Write the data_type based on file_source - fn fmt_file_source(&self, f: &mut Formatter) -> fmt::Result { - let file_source = self.source.as_any(); - let data_type = [ - ("avro", file_source.downcast_ref::().is_some()), - ("arrow", file_source.downcast_ref::().is_some()), - ("csv", file_source.downcast_ref::().is_some()), - ("json", file_source.downcast_ref::().is_some()), - #[cfg(feature = "parquet")] - ( - "parquet", - file_source.downcast_ref::().is_some(), - ), - ] - .iter() - .find(|(_, is_some)| *is_some) - .map(|(name, _)| *name) - .unwrap_or("unknown"); - - write!(f, ", file_type={}", data_type) - } - - /// Returns the base_config - pub fn base_config(&self) -> &FileScanConfig { - &self.base_config - } - - /// Returns the file_source - pub fn file_source(&self) -> &Arc { - &self.source - } - - /// Returns the `PlanProperties` of the plan - pub(crate) fn cache(&self) -> PlanProperties { - self.cache.clone() - } - - fn compute_properties( - schema: SchemaRef, - orderings: &[LexOrdering], - file_scan_config: &FileScanConfig, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - - PlanProperties::new( - eq_properties, - Self::output_partitioning_helper(file_scan_config), // Output Partitioning - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - - fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) - } - - 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 - } - - fn supports_repartition(&self) -> bool { - !(self.base_config.file_compression_type.is_compressed() - || self.base_config.new_lines_in_values - || self.source.as_any().downcast_ref::().is_some()) - } -} From 7edc80f226d85a2d9b2d1445b836d932906d975d Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Wed, 15 Jan 2025 17:54:34 +0300 Subject: [PATCH 21/39] make memory metrics default behavior --- datafusion/physical-plan/src/memory.rs | 5 ----- datafusion/physical-plan/src/source.rs | 4 +++- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 58800ecf72b6..0efe14cefbcd 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -29,7 +29,6 @@ use super::{ Statistics, }; use crate::execution_plan::{Boundedness, EmissionType}; -use crate::metrics::ExecutionPlanMetricsSet; use crate::source::{DataSource, DataSourceExec}; use arrow::datatypes::SchemaRef; @@ -116,10 +115,6 @@ impl DataSource for MemorySourceConfig { )) } - fn metrics(&self) -> ExecutionPlanMetricsSet { - ExecutionPlanMetricsSet::new() - } - fn properties(&self) -> PlanProperties { self.cache.clone() } diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs index 60ebe68c4d62..812d8df29249 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/physical-plan/src/source.rs @@ -52,7 +52,9 @@ pub trait DataSource: Send + Sync { fn fetch(&self) -> Option { None } - fn metrics(&self) -> ExecutionPlanMetricsSet; + fn metrics(&self) -> ExecutionPlanMetricsSet { + ExecutionPlanMetricsSet::new() + } fn properties(&self) -> PlanProperties; } From fe25de3b7458d0abd1979a81936367953838a49d Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 16 Jan 2025 10:20:06 +0300 Subject: [PATCH 22/39] remove unnecessary cfg check --- datafusion/core/src/datasource/data_source.rs | 60 ++++++++----------- 1 file changed, 26 insertions(+), 34 deletions(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index ad777ce05ec0..e5292c9df3f7 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -224,40 +224,32 @@ impl DataSource for FileSourceConfig { } #[cfg(feature = "parquet")] - if cfg!(feature = "parquet") { - if let Some(parquet_conf) = - self.source.as_any().downcast_ref::() - { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - let predicate_string = parquet_conf - .predicate() - .map(|p| format!(", predicate={p}")) - .unwrap_or_default(); - - let pruning_predicate_string = parquet_conf - .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(); - - return write!( - f, - "{}{}", - predicate_string, pruning_predicate_string, - ); - } + if let Some(parquet_conf) = self.source.as_any().downcast_ref::() { + return match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let predicate_string = parquet_conf + .predicate() + .map(|p| format!(", predicate={p}")) + .unwrap_or_default(); + + let pruning_predicate_string = parquet_conf + .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) } } } From 2452825f4b9db6abf26112314f25735017912a3f Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 16 Jan 2025 10:21:54 +0300 Subject: [PATCH 23/39] format code --- datafusion/core/src/datasource/data_source.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index e5292c9df3f7..3712b76ff95a 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -251,7 +251,7 @@ impl DataSource for FileSourceConfig { write!(f, "{}{}", predicate_string, pruning_predicate_string) } - } + }; } Ok(()) } From a4d5da33a798bc512df99636473d422d7a221704 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 16 Jan 2025 11:06:44 +0300 Subject: [PATCH 24/39] remove ParquetExec strings --- .../examples/advanced_parquet_index.rs | 42 ++++++------ .../physical_plan/parquet/access_plan.rs | 2 +- .../datasource/physical_plan/parquet/mod.rs | 12 ++-- .../physical_plan/parquet/opener.rs | 2 +- .../physical_plan/parquet/row_filter.rs | 4 +- datafusion/core/src/lib.rs | 64 +++++++++---------- .../src/physical_optimizer/enforce_sorting.rs | 2 +- .../combine_partial_final_agg.rs | 2 +- .../limited_distinct_aggregation.rs | 2 +- .../custom-table-providers.md | 2 +- docs/source/user-guide/explain-usage.md | 14 ++-- 11 files changed, 74 insertions(+), 74 deletions(-) diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 6a5c36778da4..3d53f8a9b2e2 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -95,29 +95,29 @@ use url::Url; /// /// # Diagram /// -/// This diagram shows how the `ParquetExec` is configured to do only a single +/// This diagram shows how the `DataSourceExec` with `ParquetConfig` 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 ParquetConfig does not re-read / @@ -136,15 +136,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 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 4679cd972c8b..f0bff9f0ca1b 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -204,12 +204,12 @@ pub use writer::plan_to_parquet; /// # 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 +/// 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 `ParquetExec` will try and reduce any provided `ParquetAccessPlan` +/// The `DataSourceExec` will try and reduce any provided `ParquetAccessPlan` /// further based on the contents of `ParquetMetadata` and other settings. /// /// ## Example of providing a ParquetAccessPlan @@ -234,12 +234,12 @@ pub use writer::plan_to_parquet; /// // 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 +/// // create a FileScanConfig to scan this file /// let file_scan_config = FileScanConfig::new(ObjectStoreUrl::local_filesystem(), schema()) /// .with_file(partitioned_file); /// // create a ParguetConfig for file opener configurations /// let source_config = Arc::new(ParquetConfig::default()); -/// // this parquet exec will not even try to read row groups 2 and 4. Additional +/// // this parquet DataSourceExec will not even try to read row groups 2 and 4. Additional /// // pruning based on predicates may also happen /// let exec = FileSourceConfig::new_exec(file_scan_config, source_config); /// ``` @@ -363,7 +363,7 @@ impl ParquetConfig { /// /// 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 ParquetExec will need to make additional IO requests to + /// too small, the ParquetConfig 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); @@ -1107,7 +1107,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 = [ "+-----+----+", diff --git a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs index 883f296f3b95..1150e6b37fc3 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs @@ -292,7 +292,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/lib.rs b/datafusion/core/src/lib.rs index e9501bd37a8a..5de41b7eba6d 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 ┃ +//! ┃ ParquetConfig) ┃ ┃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_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index f90ca3004b21..830028f9a95d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -1425,7 +1425,7 @@ mod tests { 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", 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 d3ee1b824e8c..75fd6f1dd1a2 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; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 87244f16272c..a6a80ab88591 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -18,7 +18,7 @@ //! Tests for [`LimitedDistinctAggregation`] 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 super::test_util::{parquet_exec_with_sort, schema, trim_plan_display}; use std::sync::Arc; diff --git a/docs/source/library-user-guide/custom-table-providers.md b/docs/source/library-user-guide/custom-table-providers.md index f86cea0bda95..385955bdbfac 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 [`CsvConfig`][csv] and [`ParquetConfig`][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 f68c309c2289..a553ec8eddf2 100644 --- a/docs/source/user-guide/explain-usage.md +++ b/docs/source/user-guide/explain-usage.md @@ -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. @@ -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 `ParquetConfig` 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 `ParquetConfig` 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 @@ -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` From a6c018e9aa24d647b58d910e7d27a9d6a19ad019 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 16 Jan 2025 11:39:22 +0300 Subject: [PATCH 25/39] fix documents and imports --- .../core/src/datasource/physical_plan/arrow_file.rs | 2 +- .../src/physical_optimizer/enforce_distribution.rs | 6 +++--- datafusion/core/src/physical_optimizer/test_utils.rs | 10 +++++----- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index c4822c61a447..2ea65a0ce7ff 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -68,7 +68,7 @@ impl FileSource for ArrowConfig { } } -/// The struct that implements `[FileOpener]` +/// The struct arrow that implements `[FileOpener]` trait pub struct ArrowOpener { pub object_store: Arc, pub projection: Option>, diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index f6e9e95450a0..52fcbadd5a6d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -971,7 +971,7 @@ fn add_spm_on_top(input: DistributionContext) -> DistributionContext { /// them and returns following plan: /// /// ```text -/// "DataSourceExec: 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, @@ -996,7 +996,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", -/// " DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=csv", +/// " 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: @@ -1005,7 +1005,7 @@ fn remove_dist_changing_operators( /// "CoalescePartitionsExec" /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", -/// " DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=csv", +/// " 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/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index d34106cdcad1..22a17fcb6b58 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -23,6 +23,7 @@ use std::any::Any; use std::fmt::Formatter; use std::sync::Arc; +use crate::datasource::data_source::FileSourceConfig; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use crate::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; @@ -46,21 +47,20 @@ use arrow_schema::{Schema, SchemaRef, SortOptions}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::JoinType; use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, }; -use crate::datasource::data_source::FileSourceConfig; use async_trait::async_trait; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; -use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; async fn register_current_csv( ctx: &SessionContext, From 104c428055e31549a086a160e3ef62e73a3de127 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Thu, 16 Jan 2025 11:52:45 +0300 Subject: [PATCH 26/39] fix imports --- datafusion/core/tests/fuzz_cases/merge_fuzz.rs | 3 ++- datafusion/core/tests/fuzz_cases/pruning.rs | 8 +++++--- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 4 +++- datafusion/core/tests/fuzz_cases/window_fuzz.rs | 12 ++++++------ datafusion/core/tests/parquet/page_pruning.rs | 9 +++++---- datafusion/core/tests/sql/path_partition.rs | 8 ++++---- datafusion/substrait/src/physical_plan/producer.rs | 5 ++++- 7 files changed, 29 insertions(+), 20 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 95404fa32ea8..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::MemorySourceConfig, sorts::sort_preserving_merge::SortPreservingMergeExec, }; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::memory::MemorySourceConfig; + use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; #[tokio::test] diff --git a/datafusion/core/tests/fuzz_cases/pruning.rs b/datafusion/core/tests/fuzz_cases/pruning.rs index 08157207bf3f..e648b24ce052 100644 --- a/datafusion/core/tests/fuzz_cases/pruning.rs +++ b/datafusion/core/tests/fuzz_cases/pruning.rs @@ -20,10 +20,12 @@ use std::sync::{Arc, OnceLock}; use arrow_array::{Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; use bytes::{BufMut, Bytes, BytesMut}; -use datafusion::datasource::data_source::FileSourceConfig; -use datafusion::datasource::physical_plan::ParquetConfig; use datafusion::{ - datasource::{listing::PartitionedFile, physical_plan::FileScanConfig}, + datasource::{ + data_source::FileSourceConfig, + listing::PartitionedFile, + physical_plan::{FileScanConfig, ParquetConfig}, + }, prelude::*, }; use datafusion_common::config::TableParquetOptions; diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 8a989a41e9e1..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, @@ -31,8 +33,8 @@ 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; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index e1517b556557..bf48df6a232b 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -22,6 +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::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, @@ -29,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; @@ -38,23 +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<()> { diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 85b7c1fc4054..72747649b6fa 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use crate::parquet::Unit::Page; use crate::parquet::{ContextWithParquet, Scenario}; -use std::sync::Arc; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::file_format::FileFormat; use datafusion::datasource::listing::PartitionedFile; @@ -28,14 +30,13 @@ use datafusion::execution::context::SessionState; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; +use datafusion_common::config::TableParquetOptions; 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::datasource::data_source::FileSourceConfig; -use datafusion_common::config::TableParquetOptions; use datafusion_physical_plan::source::DataSourceExec; + use futures::StreamExt; use object_store::path::Path; use object_store::ObjectMeta; diff --git a/datafusion/core/tests/sql/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs index cb57867576d0..7dd409a87294 100644 --- a/datafusion/core/tests/sql/path_partition.rs +++ b/datafusion/core/tests/sql/path_partition.rs @@ -24,6 +24,7 @@ use std::ops::Range; use std::sync::Arc; use arrow::datatypes::DataType; +use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::listing::ListingTableUrl; use datafusion::datasource::physical_plan::ParquetConfig; use datafusion::{ @@ -41,14 +42,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::datasource::data_source::FileSourceConfig; -use datafusion_expr::{col, lit, Expr, Operator}; -use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; -use datafusion_physical_plan::source::DataSourceExec; use futures::stream::{self, BoxStream}; use object_store::{ path::Path, GetOptions, GetResult, GetResultPayload, ListResult, ObjectMeta, diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 649ae21c58dd..8d5ba756e1d2 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -15,17 +15,20 @@ // 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::data_source::FileSourceConfig; use datafusion::datasource::physical_plan::ParquetConfig; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::{displayable, ExecutionPlan}; -use std::collections::HashMap; + use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; use substrait::proto::expression::MaskExpression; use substrait::proto::r#type::{ From 4af421eca1551f18553195aea02fe1145be582ba Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 20 Jan 2025 11:35:00 +0300 Subject: [PATCH 27/39] add constraints and fix tests --- datafusion/core/src/datasource/data_source.rs | 15 +++++++++---- datafusion/core/src/datasource/memory.rs | 8 +++++-- .../physical_plan/file_scan_config.rs | 3 +-- .../core/src/datasource/physical_plan/json.rs | 1 - .../enforce_distribution.rs | 7 +++--- .../src/physical_optimizer/enforce_sorting.rs | 4 +++- .../physical_optimizer/projection_pushdown.rs | 8 +++---- .../core/src/physical_optimizer/test_utils.rs | 2 +- .../physical_optimizer/sanity_checker.rs | 22 +++++++++---------- .../physical-optimizer/src/test_utils.rs | 4 ++-- datafusion/physical-plan/src/source.rs | 8 ++++++- 11 files changed, 49 insertions(+), 33 deletions(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 3712b76ff95a..be7570a20527 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -32,7 +32,7 @@ use crate::datasource::physical_plan::{ use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::Statistics; +use datafusion_common::{Constraints, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -89,11 +89,16 @@ impl FileSourceConfig { /// Initialize a new `FileSourceConfig` instance with metrics, cache, and statistics. pub fn new(base_config: FileScanConfig, file_source: Arc) -> Self { - let (projected_schema, projected_statistics, projected_output_ordering) = - base_config.project(); + let ( + projected_schema, + constraints, + projected_statistics, + projected_output_ordering, + ) = base_config.project(); let cache = Self::compute_properties( Arc::clone(&projected_schema), &projected_output_ordering, + constraints, &base_config, ); let mut metrics = ExecutionPlanMetricsSet::new(); @@ -155,10 +160,12 @@ impl FileSourceConfig { fn compute_properties( schema: SchemaRef, orderings: &[LexOrdering], + constraints: Constraints, file_scan_config: &FileScanConfig, ) -> PlanProperties { // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints); PlanProperties::new( eq_properties, diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index a765327991c3..a996990105b3 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -163,14 +163,18 @@ impl MemTable { } } - let mut exec = MemorySourceConfig::try_new_exec(&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()); } if let Some(num_partitions) = output_partitions { let exec = RepartitionExec::try_new( - exec, + Arc::new(exec), Partitioning::RoundRobinBatch(num_partitions), )?; 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 1e34b2731b12..5630b380f721 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -254,9 +254,8 @@ impl FileScanConfig { self.new_lines_in_values } - // TODO: merge project constraints /// Project the schema, constraints, and the statistics on the given column indices - pub fn project(&self) -> (SchemaRef, Statistics, Vec) { + pub fn project(&self) -> (SchemaRef, Constraints, Statistics, Vec) { if self.projection.is_none() && self.table_partition_cols.is_empty() { return ( Arc::clone(&self.file_schema), diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 2d8d298a3056..a12aa76c437b 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -34,7 +34,6 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; -use datafusion_common::Constraints; use datafusion_execution::TaskContext; use futures::{StreamExt, TryStreamExt}; diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index d01cc6d574ef..ebf86ab5faf0 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1423,10 +1423,6 @@ pub(crate) mod tests { use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; - use datafusion_physical_optimizer::output_requirements::OutputRequirements; - use datafusion_physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, repartition_exec, - }; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::ScalarValue; @@ -1437,6 +1433,9 @@ pub(crate) mod tests { }; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::output_requirements::OutputRequirements; + use datafusion_physical_optimizer::test_utils::{ + check_integrity, coalesce_partitions_exec, repartition_exec, + }; use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::PlanProperties; diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index e83bdd04fbf6..65c17372d332 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -661,7 +661,9 @@ mod tests { use crate::physical_optimizer::test_utils::{parquet_exec, parquet_exec_sorted}; use crate::physical_plan::{displayable, get_plan_string, Partitioning}; use crate::prelude::{SessionConfig, SessionContext}; - use crate::test::{data_source_exec_csv_ordered, data_source_exec_csv_sorted, stream_exec_ordered}; + use crate::test::{ + data_source_exec_csv_ordered, data_source_exec_csv_sorted, stream_exec_ordered, + }; use datafusion_physical_optimizer::test_utils::{ aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec, coalesce_partitions_exec, filter_exec, global_limit_exec, hash_join_exec, diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 9ff38ce233c8..7abf54d88d69 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -2497,8 +2497,8 @@ mod tests { 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); @@ -2506,8 +2506,8 @@ mod tests { 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(()) diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index ddfd0aff814c..9da8538eef16 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -25,11 +25,11 @@ use crate::datasource::data_source::FileSourceConfig; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{FileScanConfig, ParquetConfig}; use crate::physical_plan::ExecutionPlan; -use crate::physical_plan::DataSourceExec; use arrow_schema::SchemaRef; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_plan::source::DataSourceExec; /// Create a non sorted parquet exec pub fn parquet_exec(schema: &SchemaRef) -> Arc { diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index 538f0e443ddb..fe015f7aedd7 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -310,7 +310,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(()) @@ -332,7 +332,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); @@ -351,7 +351,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); @@ -371,7 +371,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. @@ -390,7 +390,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); @@ -431,10 +431,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); @@ -478,9 +478,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. @@ -524,10 +524,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/physical-optimizer/src/test_utils.rs b/datafusion/physical-optimizer/src/test_utils.rs index dc68f1dc9764..5453eff7ecc5 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/physical-optimizer/src/test_utils.rs @@ -39,7 +39,7 @@ 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; @@ -98,7 +98,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( diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs index 812d8df29249..8a3ed089778f 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/physical-plan/src/source.rs @@ -24,7 +24,7 @@ use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; use datafusion_common::config::ConfigOptions; -use datafusion_common::Statistics; +use datafusion_common::{Constraints, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; /// Common behaviors in Data Sources for both from Files and Memory. @@ -157,4 +157,10 @@ impl DataSourceExec { self.source = source; self } + + /// Assign constraints + pub fn with_constraints(mut self, constraints: Constraints) -> Self { + self.cache = self.cache.with_constraints(constraints); + self + } } From 91110cc580346df12b9b71a613040ae1cb590919 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 20 Jan 2025 11:42:11 +0300 Subject: [PATCH 28/39] delete redundant file --- datafusion/core/src/physical_optimizer/sanity_checker.rs | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 datafusion/core/src/physical_optimizer/sanity_checker.rs diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs deleted file mode 100644 index e69de29bb2d1..000000000000 From 6c76b3f1d2c819dfbd0c1d03cf34bbb76b825e99 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 20 Jan 2025 16:27:18 +0300 Subject: [PATCH 29/39] make metrics and statistics a part of File type specific configurations make cache a part of DataSourceExec --- datafusion/core/src/datasource/data_source.rs | 107 ++++++------------ .../core/src/datasource/file_format/arrow.rs | 5 +- .../datasource/physical_plan/arrow_file.rs | 30 ++++- .../core/src/datasource/physical_plan/avro.rs | 21 +++- .../core/src/datasource/physical_plan/csv.rs | 29 +++-- .../core/src/datasource/physical_plan/json.rs | 24 +++- .../datasource/physical_plan/parquet/mod.rs | 37 +++++- datafusion/physical-plan/src/source.rs | 18 ++- 8 files changed, 174 insertions(+), 97 deletions(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index be7570a20527..602c98eddb1a 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -31,13 +31,12 @@ use crate::datasource::physical_plan::{ }; use arrow_schema::SchemaRef; -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; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::source::{DataSource, DataSourceExec}; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, @@ -63,6 +62,12 @@ pub trait FileSource: Send + Sync { 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; } /// Holds generic file configuration, and common behaviors for file sources. @@ -72,9 +77,6 @@ pub trait FileSource: Send + Sync { pub struct FileSourceConfig { source: Arc, base_config: FileScanConfig, - metrics: ExecutionPlanMetricsSet, - projected_statistics: Statistics, - cache: PlanProperties, } impl FileSourceConfig { @@ -87,36 +89,19 @@ impl FileSourceConfig { Arc::new(DataSourceExec::new(source)) } - /// Initialize a new `FileSourceConfig` instance with metrics, cache, and statistics. + /// Initialize a new `FileSourceConfig` instance. pub fn new(base_config: FileScanConfig, file_source: Arc) -> Self { let ( - projected_schema, - constraints, + _projected_schema, + _constraints, projected_statistics, - projected_output_ordering, + _projected_output_ordering, ) = base_config.project(); - let cache = Self::compute_properties( - Arc::clone(&projected_schema), - &projected_output_ordering, - constraints, - &base_config, - ); - let mut metrics = ExecutionPlanMetricsSet::new(); - - #[cfg(feature = "parquet")] - if let Some(parquet_config) = file_source.as_any().downcast_ref::() - { - metrics = parquet_config.metrics(); - let _predicate_creation_errors = MetricBuilder::new(&metrics) - .global_counter("num_predicate_creation_errors"); - }; + let file_source = file_source.with_statistics(projected_statistics); Self { source: file_source, base_config, - metrics, - projected_statistics, - cache, } } @@ -152,11 +137,6 @@ impl FileSourceConfig { &self.source } - /// Returns the `PlanProperties` of the plan - pub(crate) fn cache(&self) -> PlanProperties { - self.cache.clone() - } - fn compute_properties( schema: SchemaRef, orderings: &[LexOrdering], @@ -181,9 +161,6 @@ impl FileSourceConfig { 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 } @@ -214,7 +191,7 @@ impl DataSource for FileSourceConfig { source.create_file_opener(object_store, &self.base_config, partition)?; let stream = - FileStream::new(&self.base_config, partition, opener, &self.metrics)?; + FileStream::new(&self.base_config, partition, opener, source.metrics())?; Ok(Box::pin(stream)) } @@ -268,54 +245,36 @@ impl DataSource for FileSourceConfig { fn repartitioned( &self, target_partitions: usize, - config: &ConfigOptions, + repartition_file_min_size: usize, exec: DataSourceExec, ) -> datafusion_common::Result>> { if !self.supports_repartition() { return Ok(None); } - 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.cache().output_ordering().is_some()) + .with_preserve_order_within_groups( + exec.properties().output_ordering().is_some(), + ) .repartition_file_groups(&self.base_config.file_groups); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - let plan = Arc::new(exec.with_source(Arc::new( - self.clone().with_file_groups(repartitioned_file_groups), - ))); - return Ok(Some(plan)); + let source = self.clone().with_file_groups(repartitioned_file_groups); + let output_partitioning = + Self::output_partitioning_helper(&source.base_config); + let plan = exec + .with_source(Arc::new(source)) + // Changing file groups may invalidate output partitioning. Update it also + .with_partitioning(output_partitioning); + return Ok(Some(Arc::new(plan))); } Ok(None) } fn statistics(&self) -> datafusion_common::Result { - #[cfg(not(feature = "parquet"))] - let stats = self.projected_statistics.clone(); - - #[cfg(feature = "parquet")] - let stats = if let Some(parquet_config) = - self.source.as_any().downcast_ref::() - { - // 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 parquet_config.pruning_predicate().is_some() - || parquet_config.page_pruning_predicate().is_some() - || (parquet_config.predicate().is_some() - && parquet_config.pushdown_filters()) - { - self.projected_statistics.clone().to_inexact() - } else { - self.projected_statistics.clone() - } - } else { - self.projected_statistics.clone() - }; - - Ok(stats) + self.source.statistics() } fn with_fetch(&self, limit: Option) -> Option> { @@ -323,9 +282,6 @@ impl DataSource for FileSourceConfig { Some(Arc::new(Self { source: Arc::clone(&self.source), base_config: config, - metrics: self.metrics.clone(), - projected_statistics: self.projected_statistics.clone(), - cache: self.cache(), })) } @@ -334,10 +290,17 @@ impl DataSource for FileSourceConfig { } fn metrics(&self) -> ExecutionPlanMetricsSet { - self.metrics.clone() + self.source.metrics().clone() } fn properties(&self) -> PlanProperties { - self.cache() + let (projected_schema, constraints, _, projected_output_ordering) = + self.base_config.project(); + Self::compute_properties( + Arc::clone(&projected_schema), + &projected_output_ordering, + constraints, + &self.base_config, + ) } } diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 6fb086ee2611..0d034e8a0d1f 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -173,7 +173,10 @@ impl FileFormat for ArrowFormat { conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { - Ok(FileSourceConfig::new_exec(conf, Arc::new(ArrowConfig {}))) + Ok(FileSourceConfig::new_exec( + conf, + Arc::new(ArrowConfig::default()), + )) } async fn create_writer_physical_plan( diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 2ea65a0ce7ff..785b3f8a5703 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -29,6 +29,9 @@ use crate::error::Result; use arrow::buffer::Buffer; use arrow_ipc::reader::FileDecoder; use arrow_schema::SchemaRef; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; + +use datafusion_common::Statistics; use futures::StreamExt; use itertools::Itertools; use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; @@ -36,7 +39,10 @@ use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; /// Arrow configuration struct that is given to DataSourceExec /// Does not hold anything special, since [`FileScanConfig`] is sufficient for arrow #[derive(Clone, Default)] -pub struct ArrowConfig {} +pub struct ArrowConfig { + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, +} impl FileSource for ArrowConfig { fn create_file_opener( @@ -56,15 +62,31 @@ impl FileSource for ArrowConfig { } fn with_batch_size(&self, _batch_size: usize) -> Arc { - Arc::new(ArrowConfig::default()) + Arc::new(Self { ..self.clone() }) } fn with_schema(&self, _schema: SchemaRef) -> Arc { - Arc::new(ArrowConfig::default()) + 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(ArrowConfig::default()) + 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")) } } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index a053296ce296..e071e10583b2 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -28,15 +28,18 @@ use crate::error::Result; use arrow::datatypes::SchemaRef; +use datafusion_common::Statistics; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use object_store::ObjectStore; -// TODO projected_constraints /// AvroConfig holds the extra configuration that is necessary for opening avro files #[derive(Clone, Default)] pub struct AvroConfig { schema: Option, batch_size: Option, projection: Option>, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, } impl AvroConfig { @@ -95,12 +98,28 @@ impl FileSource for AvroConfig { 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")) + } } #[cfg(feature = "avro")] diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 545504f34800..d02b7e9a3516 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -34,7 +34,9 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::csv; use arrow::datatypes::SchemaRef; +use datafusion_common::Statistics; use datafusion_execution::TaskContext; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; @@ -73,7 +75,7 @@ use tokio::task::JoinSet; /// )); /// let exec = FileSourceConfig::new_exec(file_scan_config, source_config); /// ``` -#[derive(Debug, Clone)] +#[derive(Debug, Clone, Default)] pub struct CsvConfig { batch_size: Option, file_schema: Option, @@ -84,21 +86,18 @@ pub struct CsvConfig { terminator: Option, escape: Option, comment: Option, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, } impl CsvConfig { /// Returns a [`CsvConfig`] pub fn new(has_header: bool, delimiter: u8, quote: u8) -> Self { Self { - batch_size: None, - file_schema: None, - file_projection: None, has_header, delimiter, quote, - terminator: None, - escape: None, - comment: None, + ..Self::default() } } @@ -240,11 +239,27 @@ impl FileSource for CsvConfig { 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")) + } } impl FileOpener for CsvOpener { diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index a12aa76c437b..3aaa01b182cd 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -35,6 +35,8 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_execution::TaskContext; +use datafusion_common::Statistics; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; @@ -71,6 +73,8 @@ impl JsonOpener { #[derive(Clone, Default)] pub struct JsonConfig { batch_size: Option, + metrics: ExecutionPlanMetricsSet, + projected_statistics: Option, } impl JsonConfig { @@ -108,11 +112,27 @@ impl FileSource for JsonConfig { } fn with_schema(&self, _schema: SchemaRef) -> Arc { - Arc::new(Self { ..*self }) + 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 }) + 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")) } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index f0bff9f0ca1b..b45585d09e9d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -51,6 +51,7 @@ use crate::datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; +use datafusion_common::Statistics; pub use metrics::ParquetFileMetrics; use opener::ParquetOpener; pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; @@ -292,6 +293,7 @@ pub struct ParquetConfig { batch_size: Option, /// Optional hint for the size of the parquet metadata metadata_size_hint: Option, + projected_statistics: Option, } impl ParquetConfig { @@ -467,11 +469,6 @@ impl ParquetConfig { self.table_parquet_options.global.pushdown_filters } - /// Return metrics - pub(crate) fn metrics(&self) -> ExecutionPlanMetricsSet { - self.metrics.clone() - } - /// 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 @@ -561,7 +558,7 @@ impl FileSource for ParquetConfig { 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(), + metrics: self.metrics().clone(), parquet_file_reader_factory, pushdown_filters: self.pushdown_filters(), reorder_filters: self.reorder_filters(), @@ -585,9 +582,37 @@ impl FileSource for ParquetConfig { 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; + 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 should_enable_page_index( diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs index 8a3ed089778f..5ddc9d66b808 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/physical-plan/src/source.rs @@ -26,6 +26,7 @@ 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::Partitioning; /// Common behaviors in Data Sources for both from Files and Memory. /// See `DataSourceExec` for physical plan implementation @@ -40,7 +41,7 @@ pub trait DataSource: Send + Sync { fn repartitioned( &self, _target_partitions: usize, - _config: &ConfigOptions, + _repartition_file_min_size: usize, _exec: DataSourceExec, ) -> datafusion_common::Result>> { Ok(None) @@ -107,8 +108,11 @@ impl ExecutionPlan for DataSourceExec { target_partitions: usize, config: &ConfigOptions, ) -> datafusion_common::Result>> { - self.source - .repartitioned(target_partitions, config, self.clone()) + self.source.repartitioned( + target_partitions, + config.optimizer.repartition_file_min_size, + self.clone(), + ) } fn execute( @@ -130,7 +134,7 @@ impl ExecutionPlan for DataSourceExec { fn with_fetch(&self, limit: Option) -> Option> { let mut source = Arc::clone(&self.source); source = source.with_fetch(limit)?; - let cache = source.properties().clone(); + let cache = self.cache.clone(); Some(Arc::new(Self { source, cache })) } @@ -163,4 +167,10 @@ impl DataSourceExec { 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 + } } From 12f0ac8624478575524816c9fd3567186c7a5ea4 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 20 Jan 2025 16:36:25 +0300 Subject: [PATCH 30/39] format code --- datafusion/core/src/datasource/physical_plan/json.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 3aaa01b182cd..8789b0060fec 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -34,8 +34,8 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; -use datafusion_execution::TaskContext; use datafusion_common::Statistics; +use datafusion_execution::TaskContext; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use futures::{StreamExt, TryStreamExt}; From 887922db3f30f01b7dbeaa3a44670f58b42f1882 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 20 Jan 2025 16:57:09 +0300 Subject: [PATCH 31/39] fix tests --- datafusion/core/tests/dataframe/mod.rs | 8 ++++---- datafusion/physical-plan/src/values.rs | 2 +- datafusion/sqllogictest/test_files/order.slt | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 954c46ab27e4..019d89b2b300 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -548,7 +548,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; @@ -592,7 +592,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; @@ -641,7 +641,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; @@ -692,7 +692,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/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index a30b8981fdd8..5207126b5c45 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -34,7 +34,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; /// Execution plan for values list based relation (produces constant rows) -#[deprecated(since = "45.0.0", note = "Use `MemoryExec::try_new_as_values` 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/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index ef932d8372fb..d95001b81b08 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -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; From 91fb10e315f6035c3447095bf9ec42d6360540d9 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Mon, 20 Jan 2025 17:28:14 +0300 Subject: [PATCH 32/39] format code --- datafusion/physical-plan/src/values.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 5207126b5c45..aa05defd0873 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -34,7 +34,10 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; /// Execution plan for values list based relation (produces constant rows) -#[deprecated(since = "45.0.0", note = "Use `MemorySourceConfig::try_new_as_values` instead")] +#[deprecated( + since = "45.0.0", + note = "Use `MemorySourceConfig::try_new_as_values` instead" +)] #[derive(Debug, Clone)] pub struct ValuesExec { /// The schema From cb8c2aee92799bd97c2d24c2b6d6924b47a4b876 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 21 Jan 2025 11:11:33 +0300 Subject: [PATCH 33/39] split repartitioning into DataSourceExec and FileSourceConfig parts --- datafusion/core/src/datasource/data_source.rs | 54 ++++++------------- .../datasource/physical_plan/parquet/mod.rs | 1 - datafusion/physical-plan/src/memory.rs | 4 ++ datafusion/physical-plan/src/source.rs | 25 +++++++-- 4 files changed, 41 insertions(+), 43 deletions(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 602c98eddb1a..503b8e30cb11 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -31,16 +31,14 @@ use crate::datasource::physical_plan::{ }; use arrow_schema::SchemaRef; -use datafusion_common::{Constraints, Statistics}; +use datafusion_common::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; use datafusion_physical_plan::source::{DataSource, DataSourceExec}; -use datafusion_physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, -}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; use itertools::Itertools; use object_store::ObjectStore; @@ -137,28 +135,21 @@ impl FileSourceConfig { &self.source } - fn compute_properties( - schema: SchemaRef, - orderings: &[LexOrdering], - constraints: Constraints, - file_scan_config: &FileScanConfig, - ) -> PlanProperties { + fn compute_properties(&self) -> PlanProperties { + let (schema, constraints, _, orderings) = self.base_config.project(); // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) - .with_constraints(constraints); + let eq_properties = + EquivalenceProperties::new_with_orderings(schema, orderings.as_slice()) + .with_constraints(constraints); PlanProperties::new( eq_properties, - Self::output_partitioning_helper(file_scan_config), // Output Partitioning + self.output_partitioning(), EmissionType::Incremental, Boundedness::Bounded, ) } - fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { - Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) - } - fn with_file_groups(mut self, file_groups: Vec>) -> Self { self.base_config.file_groups = file_groups; self @@ -246,8 +237,8 @@ impl DataSource for FileSourceConfig { &self, target_partitions: usize, repartition_file_min_size: usize, - exec: DataSourceExec, - ) -> datafusion_common::Result>> { + output_ordering: Option, + ) -> datafusion_common::Result>> { if !self.supports_repartition() { return Ok(None); } @@ -255,24 +246,20 @@ impl DataSource for FileSourceConfig { 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( - exec.properties().output_ordering().is_some(), - ) + .with_preserve_order_within_groups(output_ordering.is_some()) .repartition_file_groups(&self.base_config.file_groups); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { let source = self.clone().with_file_groups(repartitioned_file_groups); - let output_partitioning = - Self::output_partitioning_helper(&source.base_config); - let plan = exec - .with_source(Arc::new(source)) - // Changing file groups may invalidate output partitioning. Update it also - .with_partitioning(output_partitioning); - return Ok(Some(Arc::new(plan))); + return Ok(Some(Arc::new(source))); } Ok(None) } + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + } + fn statistics(&self) -> datafusion_common::Result { self.source.statistics() } @@ -294,13 +281,6 @@ impl DataSource for FileSourceConfig { } fn properties(&self) -> PlanProperties { - let (projected_schema, constraints, _, projected_output_ordering) = - self.base_config.project(); - Self::compute_properties( - Arc::clone(&projected_schema), - &projected_output_ordering, - constraints, - &self.base_config, - ) + self.compute_properties() } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index b45585d09e9d..70e10fd88982 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -1547,7 +1547,6 @@ mod tests { ); let partition_count = parquet_exec .source() - .properties() .output_partitioning() .partition_count(); assert_eq!(partition_count, 1); diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 267cc27322ce..aea1f292d467 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -120,6 +120,10 @@ impl DataSource for MemorySourceConfig { } } + fn output_partitioning(&self) -> Partitioning { + self.cache.output_partitioning().clone() + } + fn statistics(&self) -> Result { Ok(common::compute_record_batch_statistics( &self.partitions, diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs index 5ddc9d66b808..0228fdbdcd43 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/physical-plan/src/source.rs @@ -27,6 +27,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::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 @@ -42,10 +43,12 @@ pub trait DataSource: Send + Sync { &self, _target_partitions: usize, _repartition_file_min_size: usize, - _exec: DataSourceExec, - ) -> datafusion_common::Result>> { + _output_ordering: Option, + ) -> datafusion_common::Result>> { Ok(None) } + + fn output_partitioning(&self) -> Partitioning; fn statistics(&self) -> datafusion_common::Result; fn with_fetch(&self, _limit: Option) -> Option> { None @@ -108,11 +111,23 @@ impl ExecutionPlan for DataSourceExec { target_partitions: usize, config: &ConfigOptions, ) -> datafusion_common::Result>> { - self.source.repartitioned( + let source = self.source.repartitioned( target_partitions, config.optimizer.repartition_file_min_size, - self.clone(), - ) + 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 file groups 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( From 64ccad74b5d4969f5b8eb7a99a91c9e7137b672d Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 21 Jan 2025 11:46:49 +0300 Subject: [PATCH 34/39] move properties into DataSourceExec and split eq_properties and output_partitioning in DataSource trait --- datafusion/core/src/datasource/data_source.rs | 28 ++------ datafusion/physical-plan/src/memory.rs | 72 ++++--------------- datafusion/physical-plan/src/source.rs | 18 +++-- 3 files changed, 33 insertions(+), 85 deletions(-) diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 503b8e30cb11..b9f4a0ed4d03 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -35,10 +35,9 @@ use datafusion_common::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; use datafusion_physical_plan::source::{DataSource, DataSourceExec}; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; use itertools::Itertools; use object_store::ObjectStore; @@ -135,21 +134,6 @@ impl FileSourceConfig { &self.source } - fn compute_properties(&self) -> PlanProperties { - let (schema, constraints, _, orderings) = self.base_config.project(); - // Equivalence Properties - let eq_properties = - EquivalenceProperties::new_with_orderings(schema, orderings.as_slice()) - .with_constraints(constraints); - - PlanProperties::new( - eq_properties, - self.output_partitioning(), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - fn with_file_groups(mut self, file_groups: Vec>) -> Self { self.base_config.file_groups = file_groups; self @@ -260,6 +244,12 @@ impl DataSource for FileSourceConfig { Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) } + fn eq_properties(&self) -> EquivalenceProperties { + let (schema, constraints, _, orderings) = self.base_config.project(); + EquivalenceProperties::new_with_orderings(schema, orderings.as_slice()) + .with_constraints(constraints) + } + fn statistics(&self) -> datafusion_common::Result { self.source.statistics() } @@ -279,8 +269,4 @@ impl DataSource for FileSourceConfig { fn metrics(&self) -> ExecutionPlanMetricsSet { self.source.metrics().clone() } - - fn properties(&self) -> PlanProperties { - self.compute_properties() - } } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index aea1f292d467..c69c87e5a620 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -35,9 +35,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow_array::RecordBatchOptions; use arrow_schema::Schema; -use datafusion_common::{ - internal_err, plan_err, project_schema, Constraints, Result, ScalarValue, -}; +use datafusion_common::{internal_err, plan_err, project_schema, Result, ScalarValue}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; @@ -57,8 +55,6 @@ pub struct MemorySourceConfig { projected_schema: SchemaRef, /// Optional projection projection: Option>, - /// Plan Properties - cache: PlanProperties, /// Sort information: one or more equivalent orderings sort_information: Vec, /// if partition sizes should be displayed @@ -96,7 +92,8 @@ impl DataSource for MemorySourceConfig { }) .unwrap_or_default(); - let constraints = self.cache.equivalence_properties().constraints(); + let eq_properties = self.eq_properties(); + let constraints = eq_properties.constraints(); let constraints = if constraints.is_empty() { String::new() } else { @@ -121,7 +118,14 @@ impl DataSource for MemorySourceConfig { } fn output_partitioning(&self) -> Partitioning { - self.cache.output_partitioning().clone() + Partitioning::UnknownPartitioning(self.partitions.len()) + } + + fn eq_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( + Arc::clone(&self.projected_schema), + self.sort_information.as_slice(), + ) } fn statistics(&self) -> Result { @@ -131,10 +135,6 @@ impl DataSource for MemorySourceConfig { self.projection.clone(), )) } - - fn properties(&self) -> PlanProperties { - self.cache.clone() - } } impl MemorySourceConfig { @@ -146,20 +146,12 @@ impl MemorySourceConfig { 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, }) } @@ -253,40 +245,23 @@ impl MemorySourceConfig { } let partitions = vec![batches]; - let cache = Self::compute_properties( - Arc::clone(&schema), - &[], - Constraints::empty(), - &partitions, - ); let source = Self { partitions, schema: Arc::clone(&schema), projected_schema: Arc::clone(&schema), projection: None, sort_information: vec![], - cache, show_sizes: true, }; Ok(Arc::new(DataSourceExec::new(Arc::new(source)))) } - pub fn with_constraints(mut self, constraints: Constraints) -> Self { - self.cache = self.cache.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 { self.show_sizes = show_sizes; self } - /// Ref to constraints - pub fn constraints(&self) -> &Constraints { - self.cache.equivalence_properties().constraints() - } - /// Ref to partitions pub fn partitions(&self) -> &[Vec] { &self.partitions @@ -366,19 +341,12 @@ impl MemorySourceConfig { let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &self.original_schema())?; sort_information = base_eqp - .project(&projection_mapping, Arc::clone(self.properties().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( - Arc::clone(self.properties().schema()), - &self.sort_information, - ); - self.cache = self.cache.with_eq_properties(eq_properties); - Ok(self) } @@ -386,22 +354,6 @@ impl MemorySourceConfig { 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 diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs index 0228fdbdcd43..d2746e5f2e16 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/physical-plan/src/source.rs @@ -20,13 +20,14 @@ 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::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::Partitioning; +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. @@ -49,6 +50,7 @@ pub trait DataSource: Send + Sync { } fn output_partitioning(&self) -> Partitioning; + fn eq_properties(&self) -> EquivalenceProperties; fn statistics(&self) -> datafusion_common::Result; fn with_fetch(&self, _limit: Option) -> Option> { None @@ -59,7 +61,6 @@ pub trait DataSource: Send + Sync { fn metrics(&self) -> ExecutionPlanMetricsSet { ExecutionPlanMetricsSet::new() } - fn properties(&self) -> PlanProperties; } impl Debug for dyn DataSource { @@ -161,7 +162,7 @@ impl ExecutionPlan for DataSourceExec { impl DataSourceExec { pub fn new(source: Arc) -> Self { - let cache = source.properties().clone(); + let cache = Self::compute_properties(source.clone()); Self { source, cache } } @@ -172,7 +173,7 @@ impl DataSourceExec { } pub fn with_source(mut self, source: Arc) -> Self { - self.cache = source.properties(); + self.cache = Self::compute_properties(source.clone()); self.source = source; self } @@ -188,4 +189,13 @@ impl DataSourceExec { 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, + ) + } } From aa047d7e109b4ff241149e81b80748de09cdcc3c Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 21 Jan 2025 11:47:47 +0300 Subject: [PATCH 35/39] clone source with Arc --- datafusion/physical-plan/src/source.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/physical-plan/src/source.rs index d2746e5f2e16..90938c3acf84 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/physical-plan/src/source.rs @@ -162,7 +162,7 @@ impl ExecutionPlan for DataSourceExec { impl DataSourceExec { pub fn new(source: Arc) -> Self { - let cache = Self::compute_properties(source.clone()); + let cache = Self::compute_properties(Arc::clone(&source)); Self { source, cache } } @@ -173,7 +173,7 @@ impl DataSourceExec { } pub fn with_source(mut self, source: Arc) -> Self { - self.cache = Self::compute_properties(source.clone()); + self.cache = Self::compute_properties(Arc::clone(&source)); self.source = source; self } From 93888a70aa92bfdd4bdd12737637dda5db24ab3a Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 21 Jan 2025 13:37:25 +0300 Subject: [PATCH 36/39] return file type as enum and do not downcast if not necessary create fmt_extra method --- .../examples/parquet_exec_visitor.rs | 8 +- datafusion/core/src/datasource/data_source.rs | 116 ++++++++---------- .../datasource/physical_plan/arrow_file.rs | 6 +- .../core/src/datasource/physical_plan/avro.rs | 6 +- .../core/src/datasource/physical_plan/csv.rs | 10 +- .../core/src/datasource/physical_plan/json.rs | 6 +- .../datasource/physical_plan/parquet/mod.rs | 39 +++++- datafusion/core/src/test_util/parquet.rs | 7 +- datafusion/core/tests/parquet/utils.rs | 8 +- .../substrait/src/physical_plan/producer.rs | 7 +- 10 files changed, 120 insertions(+), 93 deletions(-) diff --git a/datafusion-examples/examples/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index fe281d1531a8..5496ae7a2e79 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -20,7 +20,6 @@ use std::sync::Arc; use datafusion::datasource::data_source::FileSourceConfig; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ListingOptions, PartitionedFile}; -use datafusion::datasource::physical_plan::ParquetConfig; use datafusion::execution::context::SessionContext; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::source::DataSourceExec; @@ -101,12 +100,7 @@ impl ExecutionPlanVisitor for ParquetExecVisitor { 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() - { + if file_config.file_source().file_type().is_parquet() { self.file_groups = Some(file_config.base_config().file_groups.clone()); diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index b9f4a0ed4d03..f95daa941428 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -23,11 +23,8 @@ use std::fmt::Formatter; use std::sync::Arc; use crate::datasource::listing::PartitionedFile; -#[cfg(feature = "parquet")] -use crate::datasource::physical_plan::ParquetConfig; use crate::datasource::physical_plan::{ - ArrowConfig, AvroConfig, CsvConfig, FileGroupPartitioner, FileOpener, FileScanConfig, - FileStream, JsonConfig, + FileGroupPartitioner, FileOpener, FileScanConfig, FileStream, }; use arrow_schema::SchemaRef; @@ -39,7 +36,6 @@ use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::source::{DataSource, DataSourceExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; -use itertools::Itertools; use object_store::ObjectStore; /// Common behaviors that every `FileSourceConfig` needs to implement. @@ -65,6 +61,53 @@ pub trait FileSource: Send + Sync { fn metrics(&self) -> &ExecutionPlanMetricsSet; /// Return projected statistics fn statistics(&self) -> datafusion_common::Result; + /// Returns the file type such as Arrow, Avro, Parquet, ... + fn file_type(&self) -> FileType; + /// Format FileType specific information + fn fmt_extra(&self, _t: DisplayFormatType, _f: &mut Formatter) -> fmt::Result { + Ok(()) + } +} + +/// Determines file types +pub enum FileType { + /// Arrow File + Arrow, + /// Avro File + Avro, + /// CSV File + Csv, + /// JSON File + Json, + /// Parquet File + Parquet, +} + +impl FileType { + fn to_str(&self) -> &str { + match self { + FileType::Arrow => "arrow", + FileType::Avro => "avro", + FileType::Csv => "csv", + FileType::Json => "json", + FileType::Parquet => "parquet", + } + } + + /// Is the file type avro? + pub fn is_avro(&self) -> bool { + matches!(self, FileType::Avro) + } + + /// Is the file type csv? + pub fn is_csv(&self) -> bool { + matches!(self, FileType::Csv) + } + + /// Is the file type parquet? + pub fn is_parquet(&self) -> bool { + matches!(self, FileType::Parquet) + } } /// Holds generic file configuration, and common behaviors for file sources. @@ -77,6 +120,7 @@ pub struct FileSourceConfig { } impl FileSourceConfig { + // TODO: This function should be moved into DataSourceExec once FileScanConfig and FileSourceConfig moved out of datafusion/core /// Returns a new [`DataSourceExec`] from file configurations pub fn new_exec( base_config: FileScanConfig, @@ -103,25 +147,9 @@ impl FileSourceConfig { } /// Write the data_type based on file_source - fn fmt_file_source(&self, f: &mut Formatter) -> fmt::Result { - let file_source = self.source.as_any(); - let data_type = [ - ("avro", file_source.downcast_ref::().is_some()), - ("arrow", file_source.downcast_ref::().is_some()), - ("csv", file_source.downcast_ref::().is_some()), - ("json", file_source.downcast_ref::().is_some()), - #[cfg(feature = "parquet")] - ( - "parquet", - file_source.downcast_ref::().is_some(), - ), - ] - .iter() - .find(|(_, is_some)| *is_some) - .map(|(name, _)| *name) - .unwrap_or("unknown"); - - write!(f, ", file_type={}", data_type) + fn fmt_file_source(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + write!(f, ", file_type={}", self.source.file_type().to_str())?; + self.source.fmt_extra(t, f) } /// Returns the base_config @@ -142,7 +170,7 @@ impl FileSourceConfig { fn supports_repartition(&self) -> bool { !(self.base_config.file_compression_type.is_compressed() || self.base_config.new_lines_in_values - || self.source.as_any().downcast_ref::().is_some()) + || self.source.file_type().is_avro()) } } @@ -176,43 +204,7 @@ impl DataSource for FileSourceConfig { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { self.base_config.fmt_as(t, f)?; - self.fmt_file_source(f)?; - - if let Some(csv_conf) = self.source.as_any().downcast_ref::() { - return write!(f, ", has_header={}", csv_conf.has_header); - } - - #[cfg(feature = "parquet")] - if let Some(parquet_conf) = self.source.as_any().downcast_ref::() { - return match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - let predicate_string = parquet_conf - .predicate() - .map(|p| format!(", predicate={p}")) - .unwrap_or_default(); - - let pruning_predicate_string = parquet_conf - .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) - } - }; - } - Ok(()) + self.fmt_file_source(t, f) } /// Redistribute files across partitions according to their size diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 785b3f8a5703..113dd752d0fe 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use crate::datasource::data_source::FileSource; +use crate::datasource::data_source::{FileSource, FileType}; use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, FileScanConfig, }; @@ -88,6 +88,10 @@ impl FileSource for ArrowConfig { .clone() .expect("projected_statistics must be set")) } + + fn file_type(&self) -> FileType { + FileType::Arrow + } } /// The struct arrow that implements `[FileOpener]` trait diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index e071e10583b2..ab3c464368cf 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use super::{FileOpener, FileScanConfig}; #[cfg(feature = "avro")] use crate::datasource::avro_to_arrow::Reader as AvroReader; -use crate::datasource::data_source::FileSource; +use crate::datasource::data_source::{FileSource, FileType}; use crate::error::Result; use arrow::datatypes::SchemaRef; @@ -120,6 +120,10 @@ impl FileSource for AvroConfig { .clone() .expect("projected_statistics must be set")) } + + fn file_type(&self) -> FileType { + FileType::Avro + } } #[cfg(feature = "avro")] diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index d02b7e9a3516..aeea528f91a9 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -18,12 +18,13 @@ //! 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, FileScanConfig, RangeCalculation}; -use crate::datasource::data_source::FileSource; +use crate::datasource::data_source::{FileSource, FileType}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; use crate::datasource::listing::{FileRange, ListingTableUrl}; @@ -37,6 +38,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_physical_plan::DisplayFormatType; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; @@ -260,6 +262,12 @@ impl FileSource for CsvConfig { .clone() .expect("projected_statistics must be set")) } + fn file_type(&self) -> FileType { + FileType::Csv + } + fn fmt_extra(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, ", has_header={}", self.has_header) + } } impl FileOpener for CsvOpener { diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 8789b0060fec..808348161553 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use std::task::Poll; use super::{calculate_range, FileScanConfig, RangeCalculation}; -use crate::datasource::data_source::FileSource; +use crate::datasource::data_source::{FileSource, FileType}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; use crate::datasource::listing::ListingTableUrl; @@ -134,6 +134,10 @@ impl FileSource for JsonConfig { .clone() .expect("projected_statistics must be set to call")) } + + fn file_type(&self) -> FileType { + FileType::Json + } } impl FileOpener for JsonOpener { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 70e10fd88982..9169faa6e5cd 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -18,9 +18,10 @@ //! [`ParquetConfig`] FileSourceConfig 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::data_source::{FileSource, FileType}; use crate::datasource::physical_plan::{ parquet::page_filter::PagePruningAccessPlanFilter, FileOpener, FileScanConfig, }; @@ -34,7 +35,9 @@ use crate::{ use arrow::datatypes::SchemaRef; use arrow_schema::Schema; use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_plan::DisplayFormatType; +use itertools::Itertools; use log::debug; use object_store::ObjectStore; @@ -613,6 +616,40 @@ impl FileSource for ParquetConfig { Ok(statistics) } } + + fn file_type(&self) -> FileType { + FileType::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) + } + } + } } fn should_enable_page_index( diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 31ad66c4a487..d1612871ed36 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -205,12 +205,7 @@ impl TestParquetFile { if let Some(maybe_parquet) = source.as_any().downcast_ref::() { - if maybe_parquet - .file_source() - .as_any() - .downcast_ref::() - .is_some() - { + if maybe_parquet.file_source().file_type().is_parquet() { return maybe_file.metrics(); } } diff --git a/datafusion/core/tests/parquet/utils.rs b/datafusion/core/tests/parquet/utils.rs index c9db55635219..f27822333801 100644 --- a/datafusion/core/tests/parquet/utils.rs +++ b/datafusion/core/tests/parquet/utils.rs @@ -18,7 +18,6 @@ //! Utilities for parquet tests use datafusion::datasource::data_source::FileSourceConfig; -use datafusion::datasource::physical_plan::ParquetConfig; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{accept, ExecutionPlan, ExecutionPlanVisitor}; @@ -52,12 +51,7 @@ impl ExecutionPlanVisitor for MetricsFinder { 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() - { + if file_config.file_source().file_type().is_parquet() { self.metrics = exec.metrics(); } } diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 8d5ba756e1d2..827b394c50b3 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -24,7 +24,6 @@ use crate::variation_const::{ use datafusion::arrow::datatypes::DataType; use datafusion::datasource::data_source::FileSourceConfig; -use datafusion::datasource::physical_plan::ParquetConfig; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::{displayable, ExecutionPlan}; @@ -55,11 +54,7 @@ pub fn to_substrait_rel( 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(); + let is_parquet = file_config.file_source().file_type().is_parquet(); if is_parquet { let base_config = file_config.base_config(); let mut substrait_files = vec![]; From aabcd04c2d61d0919730d90f0c8602bea0b9517c Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 21 Jan 2025 13:53:55 +0300 Subject: [PATCH 37/39] format code --- datafusion/core/src/physical_planner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 1d7578e9f5d1..cee3acc08dae 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -82,8 +82,8 @@ use datafusion_expr::{ use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; -use datafusion_physical_plan::execution_plan::InvariantLevel; 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; From 18da49430e7364dc7e08f2f745943c69c5c785d3 Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 21 Jan 2025 16:32:37 +0300 Subject: [PATCH 38/39] re-add deprecated plans in order to support backward compatibility --- .../datasource/physical_plan/arrow_file.rs | 196 +++++- .../core/src/datasource/physical_plan/avro.rs | 214 ++++++- .../core/src/datasource/physical_plan/csv.rs | 410 +++++++++++- .../core/src/datasource/physical_plan/json.rs | 207 +++++- .../datasource/physical_plan/parquet/mod.rs | 590 +++++++++++++++++- 5 files changed, 1596 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 113dd752d0fe..972d97111265 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -21,21 +21,211 @@ use std::any::Any; use std::sync::Arc; use crate::datasource::data_source::{FileSource, FileType}; +use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{ - FileMeta, FileOpenFuture, FileOpener, FileScanConfig, + FileGroupPartitioner, FileMeta, FileOpenFuture, FileOpener, FileScanConfig, }; use crate::error::Result; use arrow::buffer::Buffer; use arrow_ipc::reader::FileDecoder; use arrow_schema::SchemaRef; -use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +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; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; -use datafusion_common::Statistics; use futures::StreamExt; use itertools::Itertools; 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 { + 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 { + let ( + projected_schema, + projected_constraints, + projected_statistics, + projected_output_ordering, + ) = base_config.project(); + let cache = Self::compute_properties( + Arc::clone(&projected_schema), + &projected_output_ordering, + projected_constraints, + &base_config, + ); + Self { + 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 + } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + output_ordering: &[LexOrdering], + constraints: Constraints, + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = + EquivalenceProperties::new_with_orderings(schema, output_ordering) + .with_constraints(constraints); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + 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 + } +} + +#[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) + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for ArrowExec { + fn name(&self) -> &'static str { + "ArrowExec" + } + + 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>, + ) -> Result> { + Ok(self) + } + + /// Redistribute files across partitions according to their size + /// 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) + } + 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, + Arc::new(opener), + &self.metrics, + )?; + Ok(Box::pin(stream)) + } + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) + } + fn fetch(&self) -> Option { + self.base_config.limit + } + + 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(), + })) + } +} + /// Arrow configuration struct that is given to DataSourceExec /// Does not hold anything special, since [`FileScanConfig`] is sufficient for arrow #[derive(Clone, Default)] diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index ab3c464368cf..96fb2ac4c77a 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -18,6 +18,7 @@ //! Execution plan for reading line-delimited Avro files use std::any::Any; +use std::fmt::Formatter; use std::sync::Arc; use super::{FileOpener, FileScanConfig}; @@ -27,11 +28,177 @@ use crate::datasource::data_source::{FileSource, FileType}; use crate::error::Result; use arrow::datatypes::SchemaRef; +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::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; -use datafusion_common::Statistics; -use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; 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 { + 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 { + let ( + projected_schema, + projected_constraints, + projected_statistics, + projected_output_ordering, + ) = base_config.project(); + let cache = Self::compute_properties( + Arc::clone(&projected_schema), + &projected_output_ordering, + projected_constraints, + &base_config, + ); + Self { + 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 + } + + /// 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, + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints); + let n_partitions = file_scan_config.file_groups.len(); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(n_partitions), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +#[allow(unused, deprecated)] +impl DisplayAs for AvroExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!(f, "AvroExec: ")?; + self.base_config.fmt_as(t, f) + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for AvroExec { + fn name(&self) -> &'static str { + "AvroExec" + } + + 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>, + ) -> Result> { + Ok(self) + } + #[cfg(not(feature = "avro"))] + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + Err(crate::error::DataFusionError::NotImplemented( + "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::DeprecatedAvroConfig { + 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::DeprecatedAvroOpener { config }; + + let stream = FileStream::new( + &self.base_config, + partition, + Arc::new(opener), + &self.metrics, + )?; + Ok(Box::pin(stream)) + } + + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn fetch(&self) -> Option { + self.base_config.limit + } + + 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(), + })) + } +} + /// AvroConfig holds the extra configuration that is necessary for opening avro files #[derive(Clone, Default)] pub struct AvroConfig { @@ -134,7 +301,48 @@ mod private { use bytes::Buf; use futures::StreamExt; - use object_store::GetResultPayload; + use object_store::{GetResultPayload, ObjectStore}; + + pub struct DeprecatedAvroConfig { + pub schema: SchemaRef, + pub batch_size: usize, + pub projection: Option>, + pub object_store: Arc, + } + + impl DeprecatedAvroConfig { + fn open(&self, reader: R) -> Result> { + AvroReader::try_new( + reader, + Arc::clone(&self.schema), + self.batch_size, + self.projection.clone(), + ) + } + } + + 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, diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index aeea528f91a9..6aa09566459e 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -23,11 +23,13 @@ use std::io::{Read, Seek, SeekFrom}; use std::sync::Arc; use std::task::Poll; -use super::{calculate_range, FileScanConfig, RangeCalculation}; +use super::{ + calculate_range, FileGroupPartitioner, FileScanConfig, FileStream, RangeCalculation, +}; use crate::datasource::data_source::{FileSource, FileType}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; -use crate::datasource::listing::{FileRange, ListingTableUrl}; +use crate::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; use crate::datasource::physical_plan::file_stream::{FileOpenFuture, FileOpener}; use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; @@ -35,10 +37,14 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::csv; use arrow::datatypes::SchemaRef; -use datafusion_common::Statistics; -use datafusion_execution::TaskContext; -use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; -use datafusion_physical_plan::DisplayFormatType; +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; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; @@ -46,6 +52,398 @@ use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; use tokio::task::JoinSet; +#[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, +} + +/// Builder for [`CsvExec`]. +/// +/// See example on [`CsvExec`]. +#[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use FileSourceConfig instead")] +pub struct CsvExecBuilder { + file_scan_config: FileScanConfig, + file_compression_type: FileCompressionType, + // TODO: it seems like these format options could be reused across all the various CSV config + has_header: bool, + delimiter: u8, + quote: u8, + terminator: Option, + escape: Option, + comment: Option, + 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 { + Self { + file_scan_config, + // TODO: these defaults are duplicated from `CsvOptions` - should they be computed? + has_header: false, + delimiter: b',', + quote: b'"', + terminator: None, + escape: None, + comment: None, + newlines_in_values: false, + file_compression_type: FileCompressionType::UNCOMPRESSED, + } + } + + /// Set whether the first row defines the column names. + /// + /// The default value is `false`. + pub fn with_has_header(mut self, has_header: bool) -> Self { + self.has_header = has_header; + self + } + + /// Set the column delimeter. + /// + /// The default is `,`. + pub fn with_delimeter(mut self, delimiter: u8) -> Self { + self.delimiter = delimiter; + self + } + + /// Set the quote character. + /// + /// The default is `"`. + pub fn with_quote(mut self, quote: u8) -> Self { + self.quote = quote; + self + } + + /// Set the line terminator. If not set, the default is CRLF. + /// + /// The default is None. + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.terminator = terminator; + self + } + + /// Set the escape character. + /// + /// The default is `None` (i.e. quotes cannot be escaped). + pub fn with_escape(mut self, escape: Option) -> Self { + self.escape = escape; + self + } + + /// Set the comment character. + /// + /// The default is `None` (i.e. comments are not supported). + pub fn with_comment(mut self, comment: Option) -> Self { + self.comment = comment; + self + } + + /// Set 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 value is `false`. + pub fn with_newlines_in_values(mut self, newlines_in_values: bool) -> Self { + self.newlines_in_values = newlines_in_values; + self + } + + /// Set the file compression type. + /// + /// The default is [`FileCompressionType::UNCOMPRESSED`]. + pub fn with_file_compression_type( + mut self, + file_compression_type: FileCompressionType, + ) -> Self { + self.file_compression_type = file_compression_type; + self + } + + /// Build a [`CsvExec`]. + #[must_use] + pub fn build(self) -> CsvExec { + let Self { + file_scan_config: base_config, + file_compression_type, + has_header, + delimiter, + quote, + terminator, + escape, + comment, + newlines_in_values, + } = self; + + let ( + projected_schema, + projected_constraints, + projected_statistics, + projected_output_ordering, + ) = base_config.project(); + let cache = CsvExec::compute_properties( + projected_schema, + &projected_output_ordering, + projected_constraints, + &base_config, + ); + + CsvExec { + 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 + #[allow(clippy::too_many_arguments)] + pub fn new( + base_config: FileScanConfig, + has_header: bool, + delimiter: u8, + quote: u8, + terminator: Option, + escape: Option, + comment: Option, + newlines_in_values: bool, + file_compression_type: FileCompressionType, + ) -> Self { + CsvExecBuilder::new(base_config) + .with_has_header(has_header) + .with_delimeter(delimiter) + .with_quote(quote) + .with_terminator(terminator) + .with_escape(escape) + .with_comment(comment) + .with_newlines_in_values(newlines_in_values) + .with_file_compression_type(file_compression_type) + .build() + } + + /// Return a [`CsvExecBuilder`]. + /// + /// See example on [`CsvExec`] and [`CsvExecBuilder`] for specifying CSV table options. + pub fn builder(file_scan_config: FileScanConfig) -> CsvExecBuilder { + CsvExecBuilder::new(file_scan_config) + } + + /// Ref to the base configs + 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 + } + + /// 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.newlines_in_values + } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + /// 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, + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + 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 + } +} + +#[allow(unused, deprecated)] +impl DisplayAs for CsvExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "CsvExec: ")?; + self.base_config.fmt_as(t, f)?; + write!(f, ", has_header={}", self.has_header) + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for CsvExec { + fn name(&self) -> &'static str { + "CsvExec" + } + + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + // this is a leaf node and has no children + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + /// Redistribute files across partitions according to their size + /// See comments on [`FileGroupPartitioner`] for more detail. + /// + /// Return `None` if can't get repartitioned (empty, compressed file, or `newlines_in_values` set). + fn repartitioned( + &self, + 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) + } + + fn execute( + &self, + 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: Some(context.session_config().batch_size()), + file_schema: Some(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, + comment: self.comment, + metrics: ExecutionPlanMetricsSet::new(), + projected_statistics: None, + }); + let opener = CsvOpener { + config, + file_compression_type: self.file_compression_type.to_owned(), + object_store, + }; + let stream = FileStream::new( + &self.base_config, + partition, + Arc::new(opener), + &self.metrics, + )?; + Ok(Box::pin(stream) as SendableRecordBatchStream) + } + + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn fetch(&self) -> Option { + self.base_config.limit + } + + 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(), + })) + } +} + /// A Config for [`CsvOpener`] /// /// # Example: create a `DataSourceExec` for CSV diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 808348161553..eb76ef082ba1 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -22,11 +22,13 @@ use std::io::{BufReader, Read, Seek, SeekFrom}; use std::sync::Arc; use std::task::Poll; -use super::{calculate_range, FileScanConfig, RangeCalculation}; +use super::{ + calculate_range, FileGroupPartitioner, FileScanConfig, FileStream, RangeCalculation, +}; use crate::datasource::data_source::{FileSource, FileType}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; -use crate::datasource::listing::ListingTableUrl; +use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; use crate::datasource::physical_plan::file_stream::{FileOpenFuture, FileOpener}; use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; @@ -34,9 +36,13 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; -use datafusion_common::Statistics; -use datafusion_execution::TaskContext; -use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +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::{DisplayAs, DisplayFormatType, PlanProperties}; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; @@ -44,6 +50,197 @@ use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; 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 { + 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( + base_config: FileScanConfig, + file_compression_type: FileCompressionType, + ) -> Self { + let ( + projected_schema, + projected_constraints, + projected_statistics, + projected_output_ordering, + ) = base_config.project(); + let cache = Self::compute_properties( + projected_schema, + &projected_output_ordering, + projected_constraints, + &base_config, + ); + Self { + base_config, + projected_statistics, + metrics: ExecutionPlanMetricsSet::new(), + file_compression_type, + cache, + } + } + /// Ref to the base configs + pub fn base_config(&self) -> &FileScanConfig { + &self.base_config + } + + /// Ref to file compression type + pub fn file_compression_type(&self) -> &FileCompressionType { + &self.file_compression_type + } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + /// 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, + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + 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 + } +} + +#[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) + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for NdJsonExec { + fn name(&self) -> &'static str { + "NdJsonExec" + } + + 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>, + ) -> Result> { + Ok(self) + } + + fn repartitioned( + &self, + 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) + } + + fn execute( + &self, + 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, + Arc::new(opener), + &self.metrics, + )?; + + Ok(Box::pin(stream) as SendableRecordBatchStream) + } + + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + 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(), + })) + } +} + /// A [`FileOpener`] that opens a JSON file and yields a [`FileOpenFuture`] pub struct JsonOpener { batch_size: usize, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 9169faa6e5cd..496635d61942 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -23,7 +23,8 @@ use std::sync::Arc; use crate::datasource::data_source::{FileSource, FileType}; use crate::datasource::physical_plan::{ - parquet::page_filter::PagePruningAccessPlanFilter, FileOpener, FileScanConfig, + parquet::page_filter::PagePruningAccessPlanFilter, FileGroupPartitioner, FileOpener, + FileScanConfig, FileStream, }; use crate::{ config::TableParquetOptions, @@ -34,8 +35,10 @@ use crate::{ use arrow::datatypes::SchemaRef; use arrow_schema::Schema; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_plan::DisplayFormatType; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; use itertools::Itertools; use log::debug; @@ -50,17 +53,596 @@ mod row_filter; mod row_group_filter; mod writer; +use crate::datasource::listing::PartitionedFile; use crate::datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; -use datafusion_common::Statistics; +use datafusion_common::config::ConfigOptions; +use datafusion_common::{Constraints, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::MetricsSet; pub use metrics::ParquetFileMetrics; use opener::ParquetOpener; pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; pub use row_filter::can_expr_be_pushed_down_with_schemas; pub use writer::plan_to_parquet; +#[derive(Debug, Clone)] +#[deprecated(since = "46.0.0", note = "use DataSourceExec instead")] +/// Deprecated Execution plan replaced with FileSourceConfig and DataSourceExec +pub struct ParquetExec { + /// Base configuration for this scan + base_config: FileScanConfig, + projected_statistics: Statistics, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, + /// 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`]. +/// +/// See example on [`ParquetExec`]. +#[allow(unused, deprecated)] +pub struct ParquetExecBuilder { + file_scan_config: FileScanConfig, + predicate: Option>, + metadata_size_hint: Option, + table_parquet_options: TableParquetOptions, + parquet_file_reader_factory: Option>, + 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 { + Self::new_with_options(file_scan_config, TableParquetOptions::default()) + } + + /// Create a new builder to read the data specified in the file scan + /// configuration with the provided `TableParquetOptions`. + pub fn new_with_options( + file_scan_config: FileScanConfig, + table_parquet_options: TableParquetOptions, + ) -> Self { + Self { + file_scan_config, + predicate: None, + metadata_size_hint: None, + table_parquet_options, + parquet_file_reader_factory: None, + schema_adapter_factory: None, + } + } + + /// Update the list of files groups to read + pub fn with_file_groups(mut self, file_groups: Vec>) -> Self { + self.file_scan_config.file_groups = file_groups; + self + } + + /// Set the filter predicate when reading. + /// + /// See the "Predicate Pushdown" section of the [`ParquetExec`] documentation + /// for more details. + pub fn with_predicate(mut self, predicate: Arc) -> Self { + self.predicate = Some(predicate); + self + } + + /// 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 ParquetExec 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 + } + + /// Set the options for controlling how the ParquetExec reads parquet files. + /// + /// See also [`Self::new_with_options`] + pub fn with_table_parquet_options( + mut self, + table_parquet_options: TableParquetOptions, + ) -> Self { + self.table_parquet_options = table_parquet_options; + self + } + + /// Set optional user defined parquet file reader factory. + /// + /// You can use [`ParquetFileReaderFactory`] to more precisely control how + /// data is read from parquet files (e.g. skip re-reading metadata, coalesce + /// I/O operations, etc). + /// + /// The default reader factory reads directly from an [`ObjectStore`] + /// instance using individual I/O operations for the footer and each page. + /// + /// If a custom `ParquetFileReaderFactory` is provided, then data access + /// operations will be routed to this factory instead of [`ObjectStore`]. + /// + /// [`ObjectStore`]: object_store::ObjectStore + 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 + } + + /// 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 + } + + /// Convenience: build an `Arc`d `ParquetExec` from this builder + pub fn build_arc(self) -> Arc { + Arc::new(self.build()) + } + + /// Build a [`ParquetExec`] + #[must_use] + pub fn build(self) -> ParquetExec { + let Self { + file_scan_config, + predicate, + metadata_size_hint, + table_parquet_options, + parquet_file_reader_factory, + schema_adapter_factory, + } = self; + + let base_config = file_scan_config; + 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 { + 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, + } + } +} + +#[allow(unused, deprecated)] +impl ParquetExec { + /// Create a new Parquet reader execution plan provided file list and schema. + pub fn new( + base_config: FileScanConfig, + predicate: Option>, + metadata_size_hint: Option, + table_parquet_options: TableParquetOptions, + ) -> Self { + let mut builder = + ParquetExecBuilder::new_with_options(base_config, table_parquet_options); + if let Some(predicate) = predicate { + builder = builder.with_predicate(predicate); + } + if let Some(metadata_size_hint) = metadata_size_hint { + builder = builder.with_metadata_size_hint(metadata_size_hint); + } + builder.build() + } + /// Return a [`ParquetExecBuilder`]. + /// + /// See example on [`ParquetExec`] and [`ParquetExecBuilder`] for specifying + /// parquet table options. + pub fn builder(file_scan_config: FileScanConfig) -> ParquetExecBuilder { + ParquetExecBuilder::new(file_scan_config) + } + + /// Convert this `ParquetExec` into a builder for modification + pub fn into_builder(self) -> ParquetExecBuilder { + // 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; + ParquetExecBuilder { + file_scan_config: base_config, + predicate, + metadata_size_hint, + table_parquet_options, + parquet_file_reader_factory, + schema_adapter_factory, + } + } + /// [`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 { + 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`] + 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 + fn bloom_filter_on_read(&self) -> bool { + self.table_parquet_options.global.bloom_filter_on_read + } + /// Return the value described in [`Self::with_enable_page_index`] + fn enable_page_index(&self) -> bool { + self.table_parquet_options.global.enable_page_index + } + + fn output_partitioning_helper(file_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_config.file_groups.len()) + } + + /// 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, + file_config: &FileScanConfig, + ) -> PlanProperties { + PlanProperties::new( + EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints), + Self::output_partitioning_helper(file_config), // Output Partitioning + EmissionType::Incremental, + Boundedness::Bounded, + ) + } + + /// Updates the file groups to read and recalculates the output partitioning + /// + /// Note this function does not update statistics or other properties + /// that depend on the file groups. + fn with_file_groups_and_update_partitioning( + 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 + } +} + +#[allow(unused, deprecated)] +impl DisplayAs for ParquetExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut 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,) + } + } + } +} + +#[allow(unused, deprecated)] +impl ExecutionPlan for ParquetExec { + fn name(&self) -> &'static str { + "ParquetExec" + } + + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + // this is a leaf node and has no children + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + /// Redistribute files across partitions according to their size + /// 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))) + } + + fn execute( + &self, + 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, + Arc::new(opener), + &self.metrics, + )?; + Ok(Box::pin(stream)) + } + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + 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) + } + fn fetch(&self) -> Option { + self.base_config.limit + } + + 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(), + })) + } +} + /// Execution plan for reading one or more Parquet files. /// /// ```text From 82b5257e45cc536adb9eab487529f85c1248f9da Mon Sep 17 00:00:00 2001 From: mertak-synnada Date: Tue, 21 Jan 2025 16:40:37 +0300 Subject: [PATCH 39/39] reduce diff --- datafusion/core/src/datasource/physical_plan/json.rs | 6 ++++++ datafusion/core/src/datasource/physical_plan/parquet/mod.rs | 2 ++ 2 files changed, 8 insertions(+) diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index eb76ef082ba1..1da1a3e095aa 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -89,6 +89,7 @@ impl NdJsonExec { cache, } } + /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config @@ -228,6 +229,11 @@ impl ExecutionPlan for NdJsonExec { fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } + + fn fetch(&self) -> Option { + self.base_config.limit + } + fn with_fetch(&self, limit: Option) -> Option> { let new_config = self.base_config.clone().with_limit(limit); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 496635d61942..91ac9e887edf 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -236,6 +236,7 @@ impl ParquetExecBuilder { 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() @@ -250,6 +251,7 @@ impl ParquetExecBuilder { } }) .filter(|p| !p.always_true()); + let page_pruning_predicate = predicate .as_ref() .map(|predicate_expr| {