From fe0b1fed048192a877986245df4ac2ceb9e53c2b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 14 Feb 2024 16:57:38 +0300 Subject: [PATCH 01/15] Initial commit --- datafusion-cli/src/exec.rs | 3 +- .../examples/custom_datasource.rs | 36 ++-- datafusion/core/src/dataframe/mod.rs | 8 +- .../datasource/physical_plan/arrow_file.rs | 62 ++++-- .../core/src/datasource/physical_plan/avro.rs | 47 +++-- .../core/src/datasource/physical_plan/csv.rs | 72 ++++--- .../core/src/datasource/physical_plan/json.rs | 65 +++--- .../core/src/datasource/physical_plan/mod.rs | 15 -- .../datasource/physical_plan/parquet/mod.rs | 67 +++--- .../enforce_distribution.rs | 59 +++--- .../src/physical_optimizer/enforce_sorting.rs | 5 +- .../src/physical_optimizer/join_selection.rs | 83 +++----- .../physical_optimizer/output_requirements.rs | 42 ++-- .../physical_optimizer/pipeline_checker.rs | 53 ++--- .../replace_with_order_preserving_variants.rs | 9 +- .../src/physical_optimizer/sort_pushdown.rs | 4 +- datafusion/core/src/physical_planner.rs | 46 +++-- datafusion/core/src/test/mod.rs | 33 ++- datafusion/core/src/test_util/mod.rs | 40 ++-- datafusion/core/tests/custom_sources.rs | 69 ++++--- .../provider_filter_pushdown.rs | 52 +++-- .../tests/custom_sources_cases/statistics.rs | 43 ++-- .../tests/user_defined/user_defined_plan.rs | 54 +++-- .../physical-plan/src/aggregates/mod.rs | 192 +++++++++--------- datafusion/physical-plan/src/analyze.rs | 44 ++-- .../physical-plan/src/coalesce_batches.rs | 57 +++--- .../physical-plan/src/coalesce_partitions.rs | 52 +++-- datafusion/physical-plan/src/display.rs | 16 +- datafusion/physical-plan/src/empty.rs | 46 +++-- datafusion/physical-plan/src/explain.rs | 33 +-- datafusion/physical-plan/src/filter.rs | 96 +++++---- datafusion/physical-plan/src/insert.rs | 52 ++--- .../physical-plan/src/joins/cross_join.rs | 92 ++++----- .../physical-plan/src/joins/hash_join.rs | 181 ++++++++--------- .../src/joins/nested_loop_join.rs | 83 ++++---- .../src/joins/sort_merge_join.rs | 92 ++++----- .../src/joins/symmetric_hash_join.rs | 80 ++++---- datafusion/physical-plan/src/lib.rs | 182 ++++++++++++++--- datafusion/physical-plan/src/limit.rs | 87 ++++---- datafusion/physical-plan/src/memory.rs | 70 ++++--- .../physical-plan/src/placeholder_row.rs | 44 ++-- datafusion/physical-plan/src/projection.rs | 108 +++++----- .../physical-plan/src/recursive_query.rs | 53 ++--- .../physical-plan/src/repartition/mod.rs | 66 +++--- .../physical-plan/src/sorts/partial_sort.rs | 62 +++--- datafusion/physical-plan/src/sorts/sort.rs | 78 +++---- .../src/sorts/sort_preserving_merge.rs | 48 +++-- datafusion/physical-plan/src/streaming.rs | 91 +++++---- datafusion/physical-plan/src/test/exec.rs | 187 +++++++++++------ datafusion/physical-plan/src/union.rs | 192 ++++++++---------- datafusion/physical-plan/src/unnest.rs | 41 ++-- datafusion/physical-plan/src/values.rs | 47 +++-- .../src/windows/bounded_window_agg_exec.rs | 61 +++--- datafusion/physical-plan/src/windows/mod.rs | 10 +- .../src/windows/window_agg_exec.rs | 80 ++++---- datafusion/physical-plan/src/work_table.rs | 35 ++-- datafusion/sqllogictest/test_files/window.slt | 4 +- 57 files changed, 1982 insertions(+), 1647 deletions(-) diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 6ca8dfe927a3..06d1ea4d46b9 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -33,7 +33,6 @@ use crate::{ use datafusion::common::plan_datafusion_err; use datafusion::datasource::listing::ListingTableUrl; -use datafusion::datasource::physical_plan::is_plan_streaming; use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::{CreateExternalTable, DdlStatement, LogicalPlan}; use datafusion::physical_plan::{collect, execute_stream}; @@ -232,7 +231,7 @@ async fn exec_and_print( let df = ctx.execute_logical_plan(plan).await?; let physical_plan = df.create_physical_plan().await?; - if is_plan_streaming(&physical_plan)? { + if physical_plan.unbounded_output().is_unbounded() { let stream = execute_stream(physical_plan, task_ctx.clone())?; print_options.print_stream(stream, now).await?; } else { diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 69f9c9530e87..5ed1dfe05a14 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -28,11 +28,10 @@ use datafusion::dataframe::DataFrame; use datafusion::datasource::{provider_as_source, TableProvider, TableType}; use datafusion::error::Result; use datafusion::execution::context::{SessionState, TaskContext}; -use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::memory::MemoryStream; use datafusion::physical_plan::{ - project_schema, DisplayAs, DisplayFormatType, ExecutionPlan, - SendableRecordBatchStream, + project_schema, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, + PlanPropertiesCache, SendableRecordBatchStream, }; use datafusion::prelude::*; use datafusion_expr::{Expr, LogicalPlanBuilder}; @@ -190,6 +189,7 @@ impl TableProvider for CustomDataSource { struct CustomExec { db: CustomDataSource, projected_schema: SchemaRef, + cache: PlanPropertiesCache, } impl CustomExec { @@ -199,10 +199,28 @@ impl CustomExec { db: CustomDataSource, ) -> Self { let projected_schema = project_schema(&schema, projections).unwrap(); + let cache = PlanPropertiesCache::new_default(projected_schema.clone()); Self { db, projected_schema, + cache, } + .with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + + // Output Partitioning + let output_partitioning = + datafusion::physical_plan::Partitioning::UnknownPartitioning(1); + new_cache = new_cache.with_partitioning(output_partitioning); + + // Execution Mode + new_cache = new_cache.with_exec_mode(ExecutionMode::Bounded); + + self.cache = new_cache; + self } } @@ -217,16 +235,8 @@ impl ExecutionPlan for CustomExec { self } - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - fn output_partitioning(&self) -> datafusion::physical_plan::Partitioning { - datafusion::physical_plan::Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 237f14d2c046..784b76fc3cae 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2886,7 +2886,7 @@ mod tests { // For non-partition aware union, the output partitioning count should be the combination of all output partitions count assert!(matches!( physical_plan.output_partitioning(), - Partitioning::UnknownPartitioning(partition_count) if partition_count == default_partition_count * 2)); + Partitioning::UnknownPartitioning(partition_count) if *partition_count == default_partition_count * 2)); Ok(()) } @@ -2935,7 +2935,7 @@ mod tests { ]; assert_eq!( out_partitioning, - Partitioning::Hash(left_exprs, default_partition_count) + &Partitioning::Hash(left_exprs, default_partition_count) ); } JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { @@ -2945,13 +2945,13 @@ mod tests { ]; assert_eq!( out_partitioning, - Partitioning::Hash(right_exprs, default_partition_count) + &Partitioning::Hash(right_exprs, default_partition_count) ); } JoinType::Full => { assert!(matches!( out_partitioning, - Partitioning::UnknownPartitioning(partition_count) if partition_count == default_partition_count)); + &Partitioning::UnknownPartitioning(partition_count) if partition_count == default_partition_count)); } } } diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index f6c310fb5da1..1548d8160829 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,6 +20,7 @@ use std::any::Any; use std::sync::Arc; +use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, FileScanConfig, }; @@ -34,7 +35,8 @@ use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::Statistics; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use futures::StreamExt; use itertools::Itertools; @@ -52,6 +54,7 @@ pub struct ArrowExec { projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl ArrowExec { @@ -59,19 +62,51 @@ impl ArrowExec { pub fn new(base_config: FileScanConfig) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = PlanPropertiesCache::new_default(projected_schema.clone()); Self { base_config, projected_schema, projected_statistics, projected_output_ordering, metrics: ExecutionPlanMetricsSet::new(), + cache, } + .with_cache() } /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config } + + fn output_partitioning_helper(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ); + + // Output Partitioning + let output_partitioning = self.output_partitioning_helper(); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } + + 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.cache = self.cache.with_partitioning(output_partitioning); + self + } } impl DisplayAs for ArrowExec { @@ -90,25 +125,8 @@ impl ExecutionPlan for ArrowExec { self } - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.projected_output_ordering - .first() - .map(|ordering| ordering.as_slice()) - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -138,7 +156,7 @@ impl ExecutionPlan for ArrowExec { if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { let mut new_plan = self.clone(); - new_plan.base_config.file_groups = repartitioned_file_groups; + new_plan = new_plan.with_file_groups(repartitioned_file_groups); return Ok(Some(Arc::new(new_plan))); } Ok(None) diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index e448bf39f427..4b2521c81609 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -22,7 +22,6 @@ use std::sync::Arc; use super::FileScanConfig; use crate::error::Result; -use crate::physical_plan::expressions::PhysicalSortExpr; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, @@ -32,6 +31,7 @@ use crate::physical_plan::{ use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] @@ -43,6 +43,7 @@ pub struct AvroExec { projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl AvroExec { @@ -50,19 +51,40 @@ impl AvroExec { pub fn new(base_config: FileScanConfig) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = PlanPropertiesCache::new_default(projected_schema.clone()); Self { base_config, projected_schema, projected_statistics, projected_output_ordering, metrics: ExecutionPlanMetricsSet::new(), + cache, } + .with_cache() } /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ); + + // Output Partitioning + let output_partitioning = + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for AvroExec { @@ -81,25 +103,8 @@ impl ExecutionPlan for AvroExec { self } - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.projected_output_ordering - .first() - .map(|ordering| ordering.as_slice()) - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 04959c7904a9..831b2f40e93e 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -24,13 +24,12 @@ use std::task::Poll; use super::{calculate_range, FileGroupPartitioner, FileScanConfig, RangeCalculation}; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::listing::{FileRange, ListingTableUrl}; +use crate::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; use crate::datasource::physical_plan::file_stream::{ FileOpenFuture, FileOpener, FileStream, }; use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; -use crate::physical_plan::expressions::PhysicalSortExpr; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, @@ -41,6 +40,7 @@ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use bytes::{Buf, Bytes}; use datafusion_common::config::ConfigOptions; @@ -54,7 +54,6 @@ use tokio::task::JoinSet; pub struct CsvExec { base_config: FileScanConfig, projected_statistics: Statistics, - projected_schema: SchemaRef, projected_output_ordering: Vec, has_header: bool, delimiter: u8, @@ -64,6 +63,7 @@ pub struct CsvExec { metrics: ExecutionPlanMetricsSet, /// Compression type of the file associated with CsvExec pub file_compression_type: FileCompressionType, + cache: PlanPropertiesCache, } impl CsvExec { @@ -78,10 +78,9 @@ impl CsvExec { ) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = PlanPropertiesCache::new_default(projected_schema); Self { base_config, - projected_schema, projected_statistics, projected_output_ordering, has_header, @@ -90,7 +89,9 @@ impl CsvExec { escape, metrics: ExecutionPlanMetricsSet::new(), file_compression_type, + cache, } + .with_cache() } /// Ref to the base configs @@ -115,6 +116,36 @@ impl CsvExec { pub fn escape(&self) -> Option { self.escape } + + fn output_partitioning_helper(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ); + + // Output Partitioning + let output_partitioning = self.output_partitioning_helper(); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } + + 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.cache = self.cache.with_partitioning(output_partitioning); + self + } } impl DisplayAs for CsvExec { @@ -135,28 +166,8 @@ impl ExecutionPlan for CsvExec { self } - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) - } - - /// See comments on `impl ExecutionPlan for ParquetExec`: output order can't be - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.projected_output_ordering - .first() - .map(|ordering| ordering.as_slice()) - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -194,7 +205,7 @@ impl ExecutionPlan for CsvExec { if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { let mut new_plan = self.clone(); - new_plan.base_config.file_groups = repartitioned_file_groups; + new_plan = new_plan.with_file_groups(repartitioned_file_groups); return Ok(Some(Arc::new(new_plan))); } Ok(None) @@ -558,7 +569,6 @@ mod tests { file_compression_type.to_owned(), ); assert_eq!(13, csv.base_config.file_schema.fields().len()); - assert_eq!(3, csv.projected_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); let mut stream = csv.execute(0, task_ctx)?; @@ -624,7 +634,6 @@ mod tests { file_compression_type.to_owned(), ); assert_eq!(13, csv.base_config.file_schema.fields().len()); - assert_eq!(3, csv.projected_schema.fields().len()); assert_eq!(3, csv.schema().fields().len()); let mut stream = csv.execute(0, task_ctx)?; @@ -690,7 +699,6 @@ mod tests { file_compression_type.to_owned(), ); assert_eq!(13, csv.base_config.file_schema.fields().len()); - assert_eq!(13, csv.projected_schema.fields().len()); assert_eq!(13, csv.schema().fields().len()); let mut it = csv.execute(0, task_ctx)?; @@ -754,7 +762,6 @@ mod tests { file_compression_type.to_owned(), ); assert_eq!(14, csv.base_config.file_schema.fields().len()); - assert_eq!(14, csv.projected_schema.fields().len()); assert_eq!(14, csv.schema().fields().len()); // errors due to https://github.com/apache/arrow-datafusion/issues/4918 @@ -817,7 +824,6 @@ mod tests { file_compression_type.to_owned(), ); assert_eq!(13, csv.base_config.file_schema.fields().len()); - assert_eq!(2, csv.projected_schema.fields().len()); assert_eq!(2, csv.schema().fields().len()); let mut it = csv.execute(0, task_ctx)?; diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index c033c4b89891..5f3724cf2192 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -24,13 +24,12 @@ use std::task::Poll; use super::{calculate_range, FileGroupPartitioner, FileScanConfig, RangeCalculation}; use crate::datasource::file_format::file_compression_type::FileCompressionType; -use crate::datasource::listing::ListingTableUrl; +use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; use crate::datasource::physical_plan::file_stream::{ FileOpenFuture, FileOpener, FileStream, }; use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; -use crate::physical_plan::expressions::PhysicalSortExpr; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, @@ -43,6 +42,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{self, GetOptions}; use object_store::{GetResultPayload, ObjectStore}; @@ -54,11 +54,11 @@ use tokio::task::JoinSet; pub struct NdJsonExec { base_config: FileScanConfig, projected_statistics: Statistics, - projected_schema: SchemaRef, projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, file_compression_type: FileCompressionType, + cache: PlanPropertiesCache, } impl NdJsonExec { @@ -69,21 +69,52 @@ impl NdJsonExec { ) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = PlanPropertiesCache::new_default(projected_schema); Self { base_config, - projected_schema, projected_statistics, projected_output_ordering, metrics: ExecutionPlanMetricsSet::new(), file_compression_type, + cache, } + .with_cache() } /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config } + + fn output_partitioning_helper(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ); + + // Output Partitioning + let output_partitioning = self.output_partitioning_helper(); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } + + 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.cache = self.cache.with_partitioning(output_partitioning); + self + } } impl DisplayAs for NdJsonExec { @@ -101,26 +132,8 @@ impl ExecutionPlan for NdJsonExec { fn as_any(&self) -> &dyn Any { self } - - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.projected_output_ordering - .first() - .map(|ordering| ordering.as_slice()) - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -151,7 +164,7 @@ impl ExecutionPlan for NdJsonExec { if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { let mut new_plan = self.clone(); - new_plan.base_config.file_groups = repartitioned_file_groups; + new_plan = new_plan.with_file_groups(repartitioned_file_groups); return Ok(Some(Arc::new(new_plan))); } diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 11eb9e7867bb..365a7a5664cb 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -71,7 +71,6 @@ use arrow::{ use datafusion_common::{file_options::FileTypeWriterOptions, plan_err}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_plan::ExecutionPlan; use log::debug; use object_store::ObjectMeta; @@ -506,20 +505,6 @@ fn get_projected_output_ordering( all_orderings } -/// Get output (un)boundedness information for the given `plan`. -pub fn is_plan_streaming(plan: &Arc) -> Result { - if plan.children().is_empty() { - plan.unbounded_output(&[]) - } else { - let children_unbounded_output = plan - .children() - .iter() - .map(is_plan_streaming) - .collect::>>(); - plan.unbounded_output(&children_unbounded_output?) - } -} - /// Represents the possible outcomes of a range calculation. /// /// This enum is used to encapsulate the result of calculating the range of diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 171f7cdc59a4..d12fb5fafbf7 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -22,6 +22,7 @@ use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; +use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::file_stream::{ FileOpenFuture, FileOpener, FileStream, }; @@ -44,9 +45,8 @@ use crate::{ use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; -use datafusion_physical_expr::{ - EquivalenceProperties, LexOrdering, PhysicalExpr, PhysicalSortExpr, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use bytes::Bytes; use futures::future::BoxFuture; @@ -89,7 +89,6 @@ pub struct ParquetExec { /// Base configuration for this scan base_config: FileScanConfig, projected_statistics: Statistics, - projected_schema: SchemaRef, projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, @@ -103,6 +102,7 @@ pub struct ParquetExec { metadata_size_hint: Option, /// Optional user defined parquet file reader factory parquet_file_reader_factory: Option>, + cache: PlanPropertiesCache, } impl ParquetExec { @@ -150,14 +150,13 @@ impl ParquetExec { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = PlanPropertiesCache::new_default(projected_schema); Self { pushdown_filters: None, reorder_filters: None, enable_page_index: None, enable_bloom_filter: None, base_config, - projected_schema, projected_statistics, projected_output_ordering, metrics, @@ -166,7 +165,9 @@ impl ParquetExec { page_pruning_predicate, metadata_size_hint, parquet_file_reader_factory: None, + cache, } + .with_cache() } /// Ref to the base configs @@ -260,6 +261,36 @@ impl ParquetExec { self.enable_bloom_filter .unwrap_or(config_options.execution.parquet.bloom_filter_enabled) } + + fn output_partitioning_helper(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ); + + // Output Partitioning + let output_partitioning = self.output_partitioning_helper(); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } + + 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.cache = self.cache.with_partitioning(output_partitioning); + self + } } impl DisplayAs for ParquetExec { @@ -306,8 +337,8 @@ impl ExecutionPlan for ParquetExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.projected_schema) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -315,24 +346,6 @@ impl ExecutionPlan for ParquetExec { vec![] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.projected_output_ordering - .first() - .map(|ordering| ordering.as_slice()) - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ) - } - fn with_new_children( self: Arc, _: Vec>, @@ -356,7 +369,7 @@ impl ExecutionPlan for ParquetExec { let mut new_plan = self.clone(); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - new_plan.base_config.file_groups = repartitioned_file_groups; + new_plan = new_plan.with_file_groups(repartitioned_file_groups); } Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 4f8806a68592..de77f53e49b0 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -54,7 +54,6 @@ use datafusion_physical_expr::{ PhysicalExprRef, PhysicalSortRequirement, }; use datafusion_physical_plan::sorts::sort::SortExec; -use datafusion_physical_plan::unbounded_output; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; use itertools::izip; @@ -433,7 +432,7 @@ where right_keys, }, new_positions, - )) = try_reorder(join_key_pairs.clone(), parent_required, &eq_properties) + )) = try_reorder(join_key_pairs.clone(), parent_required, eq_properties) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -616,8 +615,8 @@ pub(crate) fn reorder_join_keys_to_inputs( join_key_pairs, Some(left.output_partitioning()), Some(right.output_partitioning()), - &left.equivalence_properties(), - &right.equivalence_properties(), + left.equivalence_properties(), + right.equivalence_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -655,8 +654,8 @@ pub(crate) fn reorder_join_keys_to_inputs( join_key_pairs, Some(left.output_partitioning()), Some(right.output_partitioning()), - &left.equivalence_properties(), - &right.equivalence_properties(), + left.equivalence_properties(), + right.equivalence_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -682,14 +681,14 @@ pub(crate) fn reorder_join_keys_to_inputs( /// Reorder the current join keys ordering based on either left partition or right partition fn reorder_current_join_keys( join_keys: JoinKeyPairs, - left_partition: Option, - right_partition: Option, + left_partition: Option<&Partitioning>, + right_partition: Option<&Partitioning>, left_equivalence_properties: &EquivalenceProperties, right_equivalence_properties: &EquivalenceProperties, ) -> Option<(JoinKeyPairs, Vec)> { - match (left_partition, right_partition.clone()) { + match (left_partition, right_partition) { (Some(Partitioning::Hash(left_exprs, _)), _) => { - try_reorder(join_keys.clone(), &left_exprs, left_equivalence_properties) + try_reorder(join_keys.clone(), left_exprs, left_equivalence_properties) .or_else(|| { reorder_current_join_keys( join_keys, @@ -701,7 +700,7 @@ fn reorder_current_join_keys( }) } (_, Some(Partitioning::Hash(right_exprs, _))) => { - try_reorder(join_keys, &right_exprs, right_equivalence_properties) + try_reorder(join_keys, right_exprs, right_equivalence_properties) } _ => None, } @@ -887,7 +886,7 @@ fn add_hash_on_top( .plan .output_partitioning() .satisfy(Distribution::HashPartitioned(hash_exprs.clone()), || { - input.plan.equivalence_properties() + input.plan.equivalence_properties().clone() }); // Add hash repartitioning when: @@ -1077,7 +1076,7 @@ fn ensure_distribution( let enable_round_robin = config.optimizer.enable_round_robin_repartition; let repartition_file_scans = config.optimizer.repartition_file_scans; let batch_size = config.execution.batch_size; - let is_unbounded = unbounded_output(&dist_context.plan); + let is_unbounded = dist_context.plan.unbounded_output().is_unbounded(); // Use order preserving variants either of the conditions true // - it is desired according to config // - when plan is unbounded @@ -1344,6 +1343,7 @@ pub(crate) mod tests { expressions, expressions::binary, expressions::lit, expressions::Column, LexOrdering, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; + use datafusion_physical_plan::PlanPropertiesCache; /// Models operators like BoundedWindowExec that require an input /// ordering but is easy to construct @@ -1351,22 +1351,41 @@ pub(crate) mod tests { struct SortRequiredExec { input: Arc, expr: LexOrdering, + cache: PlanPropertiesCache, } impl SortRequiredExec { fn new(input: Arc) -> Self { let expr = input.output_ordering().unwrap_or(&[]).to_vec(); - Self { input, expr } + Self::new_with_requirement(input, expr) } fn new_with_requirement( input: Arc, requirement: Vec, ) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); Self { input, expr: requirement, + cache, } + .with_cache() + } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = self.input.equivalence_properties().clone(); + + // Output Partitioning + let output_partitioning = self.input.output_partitioning().clone(); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self } } @@ -1389,22 +1408,14 @@ pub(crate) mod tests { self } - fn schema(&self) -> SchemaRef { - self.input.schema() - } - - fn output_partitioning(&self) -> crate::physical_plan::Partitioning { - self.input.output_partitioning() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn benefits_from_input_partitioning(&self) -> Vec { vec![false] } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input.output_ordering() - } - fn children(&self) -> Vec> { vec![self.input.clone()] } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 9b76af2dbb1f..ea83e2b6b38f 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -64,7 +64,6 @@ use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; -use datafusion_physical_plan::unbounded_output; use itertools::izip; @@ -208,7 +207,7 @@ fn replace_with_partial_sort( let plan_any = plan.as_any(); if let Some(sort_plan) = plan_any.downcast_ref::() { let child = sort_plan.children()[0].clone(); - if !unbounded_output(&child) { + if !child.unbounded_output().is_unbounded() { return Ok(plan); } @@ -574,7 +573,7 @@ fn remove_corresponding_sort_from_sub_plan( { node.plan = Arc::new(RepartitionExec::try_new( node.children[0].plan.clone(), - repartition.output_partitioning(), + repartition.output_partitioning().clone(), )?) as _; } }; diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 02626056f6cc..8e3d14543b3f 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -27,9 +27,6 @@ use std::sync::Arc; use crate::config::ConfigOptions; use crate::error::Result; -use crate::physical_optimizer::pipeline_checker::{ - children_unbounded, PipelineStatePropagator, -}; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; use crate::physical_plan::joins::{ @@ -231,7 +228,6 @@ impl PhysicalOptimizerRule for JoinSelection { plan: Arc, config: &ConfigOptions, ) -> Result> { - let pipeline = PipelineStatePropagator::new_default(plan); // First, we make pipeline-fixing modifications to joins so as to accommodate // unbounded inputs. Each pipeline-fixing subrule, which is a function // of type `PipelineFixerSubrule`, takes a single [`PipelineStatePropagator`] @@ -241,7 +237,7 @@ impl PhysicalOptimizerRule for JoinSelection { Box::new(hash_join_convert_symmetric_subrule), Box::new(hash_join_swap_subrule), ]; - let state = pipeline.transform_up(&|p| apply_subrules(p, &subrules, config))?; + let new_plan = plan.transform_up(&|p| apply_subrules(p, &subrules, config))?; // Next, we apply another subrule that tries to optimize joins using any // statistics their inputs might have. // - For a hash join with partition mode [`PartitionMode::Auto`], we will @@ -256,7 +252,7 @@ impl PhysicalOptimizerRule for JoinSelection { let config = &config.optimizer; let collect_threshold_byte_size = config.hash_join_single_partition_threshold; let collect_threshold_num_rows = config.hash_join_single_partition_threshold_rows; - state.plan.transform_up(&|plan| { + new_plan.transform_up(&|plan| { statistical_join_selection_subrule( plan, collect_threshold_byte_size, @@ -446,7 +442,7 @@ fn statistical_join_selection_subrule( /// Pipeline-fixing join selection subrule. pub type PipelineFixerSubrule = - dyn Fn(PipelineStatePropagator, &ConfigOptions) -> Result; + dyn Fn(Arc, &ConfigOptions) -> Result>; /// Converts a hash join to a symmetric hash join in the case of infinite inputs on both sides. /// @@ -464,16 +460,13 @@ pub type PipelineFixerSubrule = /// it returns `None`. If applicable, it returns `Some(Ok(...))` with the modified pipeline state, /// or `Some(Err(...))` if an error occurs during the transformation. fn hash_join_convert_symmetric_subrule( - mut input: PipelineStatePropagator, + input: Arc, config_options: &ConfigOptions, -) -> Result { +) -> Result> { // Check if the current plan node is a HashJoinExec. - if let Some(hash_join) = input.plan.as_any().downcast_ref::() { - // Determine if left and right children are unbounded. - let ub_flags = children_unbounded(&input); - let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]); - // Update the unbounded flag of the input. - input.data = left_unbounded || right_unbounded; + if let Some(hash_join) = input.as_any().downcast_ref::() { + let left_unbounded = hash_join.left.unbounded_output().is_unbounded(); + let right_unbounded = hash_join.right.unbounded_output().is_unbounded(); // Process only if both left and right sides are unbounded. if left_unbounded && right_unbounded { // Determine the partition mode based on configuration. @@ -550,10 +543,7 @@ fn hash_join_convert_symmetric_subrule( right_order, mode, ) - .map(|exec| { - input.plan = Arc::new(exec) as _; - input - }); + .map(|exec| Arc::new(exec) as _); } } Ok(input) @@ -601,13 +591,12 @@ fn hash_join_convert_symmetric_subrule( /// /// ``` fn hash_join_swap_subrule( - mut input: PipelineStatePropagator, + mut input: Arc, _config_options: &ConfigOptions, -) -> Result { - if let Some(hash_join) = input.plan.as_any().downcast_ref::() { - let ub_flags = children_unbounded(&input); - let (left_unbounded, right_unbounded) = (ub_flags[0], ub_flags[1]); - input.data = left_unbounded || right_unbounded; +) -> Result> { + if let Some(hash_join) = input.as_any().downcast_ref::() { + let left_unbounded = hash_join.left.unbounded_output().is_unbounded(); + let right_unbounded = hash_join.right.unbounded_output().is_unbounded(); if left_unbounded && !right_unbounded && matches!( @@ -618,7 +607,7 @@ fn hash_join_swap_subrule( | JoinType::LeftAnti ) { - input.plan = swap_join_according_to_unboundedness(hash_join)?; + input = swap_join_according_to_unboundedness(hash_join)?; } } Ok(input) @@ -654,23 +643,13 @@ fn swap_join_according_to_unboundedness( /// Apply given `PipelineFixerSubrule`s to a given plan. This plan, along with /// auxiliary boundedness information, is in the `PipelineStatePropagator` object. fn apply_subrules( - mut input: PipelineStatePropagator, + mut input: Arc, subrules: &Vec>, config_options: &ConfigOptions, -) -> Result> { +) -> Result>> { for subrule in subrules { input = subrule(input, config_options)?; } - input.data = input - .plan - .unbounded_output(&children_unbounded(&input)) - // Treat the case where an operator can not run on unbounded data as - // if it can and it outputs unbounded data. Do not raise an error yet. - // Such operators may be fixed, adjusted or replaced later on during - // optimization passes -- sorts may be removed, windows may be adjusted - // etc. If this doesn't happen, the final `PipelineChecker` rule will - // catch this and raise an error anyway. - .unwrap_or(true); Ok(Transformed::Yes(input)) } @@ -680,7 +659,6 @@ mod tests_statistical { use super::*; use crate::{ - physical_optimizer::test_utils::check_integrity, physical_plan::{ displayable, joins::PartitionMode, ColumnStatistics, Statistics, }, @@ -829,19 +807,17 @@ mod tests_statistical { } pub(crate) fn crosscheck_plans(plan: Arc) -> Result<()> { - let pipeline = PipelineStatePropagator::new_default(plan); let subrules: Vec> = vec![ Box::new(hash_join_convert_symmetric_subrule), Box::new(hash_join_swap_subrule), ]; - let state = pipeline - .transform_up(&|p| apply_subrules(p, &subrules, &ConfigOptions::new())) - .and_then(check_integrity)?; + let new_plan = + plan.transform_up(&|p| apply_subrules(p, &subrules, &ConfigOptions::new()))?; // TODO: End state payloads will be checked here. let config = ConfigOptions::new().optimizer; let collect_left_threshold = config.hash_join_single_partition_threshold; let collect_threshold_num_rows = config.hash_join_single_partition_threshold_rows; - let _ = state.plan.transform_up(&|plan| { + let _ = new_plan.transform_up(&|plan| { statistical_join_selection_subrule( plan, collect_left_threshold, @@ -1404,7 +1380,6 @@ mod hash_join_tests { use arrow::record_batch::RecordBatch; use datafusion_common::utils::DataPtr; use datafusion_common::JoinType; - use datafusion_physical_plan::empty::EmptyExec; use std::sync::Arc; struct TestCase { @@ -1772,18 +1747,8 @@ mod hash_join_tests { false, )?); - let left_child = Arc::new(EmptyExec::new(Arc::new(Schema::empty()))); - let right_child = Arc::new(EmptyExec::new(Arc::new(Schema::empty()))); - let children = vec![ - PipelineStatePropagator::new(left_child, left_unbounded, vec![]), - PipelineStatePropagator::new(right_child, right_unbounded, vec![]), - ]; - let initial_hash_join_state = - PipelineStatePropagator::new(join.clone(), false, children); - - let optimized_hash_join = - hash_join_swap_subrule(initial_hash_join_state, &ConfigOptions::new())?; - let optimized_join_plan = optimized_hash_join.plan; + let optimized_join_plan = + hash_join_swap_subrule(join.clone(), &ConfigOptions::new())?; // If swap did happen let projection_added = optimized_join_plan.as_any().is::(); @@ -1814,12 +1779,12 @@ mod hash_join_tests { assert_eq!( ( t.case.as_str(), - if left.unbounded_output(&[])? { + if left.unbounded_output().is_unbounded() { SourceType::Unbounded } else { SourceType::Bounded }, - if right.unbounded_output(&[])? { + if right.unbounded_output().is_unbounded() { SourceType::Unbounded } else { SourceType::Bounded diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 4d03840d3dd3..52a1afdbd1ba 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -28,14 +28,12 @@ use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; -use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Result, Statistics}; -use datafusion_physical_expr::{ - Distribution, LexRequirement, PhysicalSortExpr, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::PlanPropertiesCache; /// This rule either adds or removes [`OutputRequirements`]s to/from the physical /// plan according to its `mode` attribute, which is set by the constructors @@ -92,6 +90,7 @@ pub(crate) struct OutputRequirementExec { input: Arc, order_requirement: Option, dist_requirement: Distribution, + cache: PlanPropertiesCache, } impl OutputRequirementExec { @@ -100,16 +99,34 @@ impl OutputRequirementExec { requirements: Option, dist_requirement: Distribution, ) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); Self { input, order_requirement: requirements, dist_requirement, + cache, } + .with_cache() } pub(crate) fn input(&self) -> Arc { self.input.clone() } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = self.input.equivalence_properties().clone(); + + // Output Partitioning + let output_partitioning = self.input.output_partitioning().clone(); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for OutputRequirementExec { @@ -127,12 +144,8 @@ impl ExecutionPlan for OutputRequirementExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() - } - - fn output_partitioning(&self) -> crate::physical_plan::Partitioning { - self.input.output_partitioning() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn benefits_from_input_partitioning(&self) -> Vec { @@ -143,10 +156,6 @@ impl ExecutionPlan for OutputRequirementExec { vec![self.dist_requirement.clone()] } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input.output_ordering() - } - fn maintains_input_order(&self) -> Vec { vec![true] } @@ -159,11 +168,6 @@ impl ExecutionPlan for OutputRequirementExec { vec![self.order_requirement.clone()] } - fn unbounded_output(&self, children: &[bool]) -> Result { - // Has a single child - Ok(children[0]) - } - fn with_new_children( self: Arc, mut children: Vec>, diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index bb0665c10bcc..780a09f24380 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -28,10 +28,9 @@ use crate::physical_plan::ExecutionPlan; use datafusion_common::config::OptimizerOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{plan_err, DataFusionError}; +use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; -use datafusion_physical_plan::tree_node::PlanContext; /// The PipelineChecker rule rejects non-runnable query plans that use /// pipeline-breaking operators on infinite input(s). @@ -51,10 +50,9 @@ impl PhysicalOptimizerRule for PipelineChecker { plan: Arc, config: &ConfigOptions, ) -> Result> { - let pipeline = PipelineStatePropagator::new_default(plan); - let state = pipeline - .transform_up(&|p| check_finiteness_requirements(p, &config.optimizer))?; - Ok(state.plan) + let plan = + plan.transform_up(&|p| check_finiteness_requirements(p, &config.optimizer))?; + Ok(plan) } fn name(&self) -> &str { @@ -66,21 +64,13 @@ impl PhysicalOptimizerRule for PipelineChecker { } } -/// This object propagates the [`ExecutionPlan`] pipelining information. -pub type PipelineStatePropagator = PlanContext; - -/// Collects unboundedness flags of all the children of the plan in `pipeline`. -pub fn children_unbounded(pipeline: &PipelineStatePropagator) -> Vec { - pipeline.children.iter().map(|c| c.data).collect() -} - /// This function propagates finiteness information and rejects any plan with /// pipeline-breaking operators acting on infinite inputs. pub fn check_finiteness_requirements( - mut input: PipelineStatePropagator, + input: Arc, optimizer_options: &OptimizerOptions, -) -> Result> { - if let Some(exec) = input.plan.as_any().downcast_ref::() { +) -> Result>> { + if let Some(exec) = input.as_any().downcast_ref::() { if !(optimizer_options.allow_symmetric_joins_without_pruning || (exec.check_if_order_information_available()? && is_prunable(exec))) { @@ -88,13 +78,14 @@ pub fn check_finiteness_requirements( the 'allow_symmetric_joins_without_pruning' configuration flag"); } } - input - .plan - .unbounded_output(&children_unbounded(&input)) - .map(|value| { - input.data = value; - Transformed::Yes(input) - }) + if !input.unbounded_output().is_executable() { + Err(plan_datafusion_err!( + "Cannot execute pipeline breaking queries, operator: {:?}", + input + )) + } else { + Ok(Transformed::No(input)) + } } /// This function returns whether a given symmetric hash join is amenable to @@ -141,7 +132,7 @@ mod sql_tests { sql: "SELECT t2.c1 FROM left as t1 LEFT JOIN right as t2 ON t1.c1 = t2.c1" .to_string(), cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "Join Error".to_string(), + error_operator: "operator: HashJoinExec".to_string(), }; case.run().await?; @@ -166,7 +157,7 @@ mod sql_tests { sql: "SELECT t2.c1 FROM left as t1 RIGHT JOIN right as t2 ON t1.c1 = t2.c1" .to_string(), cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "Join Error".to_string(), + error_operator: "operator: HashJoinExec".to_string(), }; case.run().await?; @@ -216,7 +207,7 @@ mod sql_tests { sql: "SELECT t2.c1 FROM left as t1 FULL JOIN right as t2 ON t1.c1 = t2.c1" .to_string(), cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "Join Error".to_string(), + error_operator: "operator: HashJoinExec".to_string(), }; case.run().await?; @@ -236,7 +227,7 @@ mod sql_tests { let case = QueryCase { sql: "SELECT c1, MIN(c4) FROM test GROUP BY c1".to_string(), cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "Aggregate Error".to_string(), + error_operator: "operator: AggregateExec".to_string(), }; case.run().await?; @@ -260,7 +251,7 @@ mod sql_tests { FROM test LIMIT 5".to_string(), cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "Sort Error".to_string() + error_operator: "operator: SortExec".to_string() }; case.run().await?; @@ -283,7 +274,7 @@ mod sql_tests { SUM(c9) OVER(ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 FROM test".to_string(), cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "Sort Error".to_string() + error_operator: "operator: SortExec".to_string() }; case.run().await?; Ok(()) @@ -315,7 +306,7 @@ mod sql_tests { Arc::new(test3), Arc::new(test4), ], - error_operator: "Cross Join Error".to_string(), + error_operator: "operator: CrossJoinExec".to_string(), }; case.run().await?; 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 bc9bd0010dc5..a03cb40e8a13 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 @@ -31,7 +31,6 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::Transformed; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::tree_node::PlanContext; -use datafusion_physical_plan::unbounded_output; use itertools::izip; @@ -120,7 +119,7 @@ fn plan_with_order_preserving_variants( // When a `RepartitionExec` doesn't preserve ordering, replace it with // a sort-preserving variant if appropriate: let child = sort_input.children[0].plan.clone(); - let partitioning = sort_input.plan.output_partitioning(); + let partitioning = sort_input.plan.output_partitioning().clone(); sort_input.plan = Arc::new( RepartitionExec::try_new(child, partitioning)?.with_preserve_order(), ) as _; @@ -176,7 +175,7 @@ fn plan_with_order_breaking_variants( // When a `RepartitionExec` preserves ordering, replace it with a // non-sort-preserving variant: let child = sort_input.children[0].plan.clone(); - let partitioning = plan.output_partitioning(); + let partitioning = plan.output_partitioning().clone(); sort_input.plan = Arc::new(RepartitionExec::try_new(child, partitioning)?) as _; } else if is_sort_preserving_merge(plan) { // Replace `SortPreservingMergeExec` with a `CoalescePartitionsExec`: @@ -241,8 +240,8 @@ pub(crate) fn replace_with_order_preserving_variants( // For unbounded cases, we replace with the order-preserving variant in any // case, as doing so helps fix the pipeline. Also replace if config allows. - let use_order_preserving_variant = - config.optimizer.prefer_existing_sort || unbounded_output(&requirements.plan); + let use_order_preserving_variant = config.optimizer.prefer_existing_sort + || !requirements.plan.unbounded_output().is_executable(); // Create an alternate plan with order-preserving variants: let mut alternate_plan = plan_with_order_preserving_variants( diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 3413486c6b46..0efa908cf5fc 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -184,7 +184,7 @@ fn pushdown_requirement_to_children( } else if is_sort_preserving_merge(plan) { let new_ordering = PhysicalSortRequirement::to_sort_exprs(parent_required.to_vec()); - let mut spm_eqs = plan.equivalence_properties(); + let mut spm_eqs = plan.equivalence_properties().clone(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_eqs = spm_eqs.with_reorder(new_ordering); // Do not push-down through SortPreservingMergeExec when @@ -262,7 +262,7 @@ fn try_pushdown_requirements_to_join( &smj.maintains_input_order(), Some(probe_side), ); - let mut smj_eqs = smj.equivalence_properties(); + let mut smj_eqs = smj.equivalence_properties().clone(); // smj will have this ordering when its input changes. smj_eqs = smj_eqs.with_reorder(new_output_ordering.unwrap_or_default()); let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 463d0cde8282..165c8818f403 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1995,6 +1995,7 @@ mod tests { col, lit, sum, Extension, GroupingSet, LogicalPlanBuilder, UserDefinedLogicalNodeCore, }; + use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use fmt::Debug; use std::collections::HashMap; use std::convert::TryFrom; @@ -2561,7 +2562,28 @@ mod tests { #[derive(Debug)] struct NoOpExecutionPlan { - schema: SchemaRef, + cache: PlanPropertiesCache, + } + + impl NoOpExecutionPlan { + fn new(schema: SchemaRef) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); + Self { cache }.with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + new_cache = new_cache.with_exec_mode(ExecutionMode::Bounded); + + self.cache = new_cache; + self + } } impl DisplayAs for NoOpExecutionPlan { @@ -2580,16 +2602,8 @@ mod tests { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -2627,13 +2641,9 @@ mod tests { _physical_inputs: &[Arc], _session_state: &SessionState, ) -> Result>> { - Ok(Some(Arc::new(NoOpExecutionPlan { - schema: SchemaRef::new(Schema::new(vec![Field::new( - "b", - DataType::Int32, - false, - )])), - }))) + Ok(Some(Arc::new(NoOpExecutionPlan::new(SchemaRef::new( + Schema::new(vec![Field::new("b", DataType::Int32, false)]), + ))))) } } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index ed5aa15e291b..80560fd8d107 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -44,7 +44,9 @@ use datafusion_common::{DataFusionError, FileType, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionMode, PlanPropertiesCache, +}; #[cfg(feature = "compression")] use bzip2::write::BzEncoder; @@ -365,6 +367,7 @@ pub fn csv_exec_ordered( pub struct StatisticsExec { stats: Statistics, schema: Arc, + cache: PlanPropertiesCache, } impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { @@ -372,10 +375,26 @@ impl StatisticsExec { stats.column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); + let cache = PlanPropertiesCache::new_default(Arc::new(schema.clone())); Self { stats, schema: Arc::new(schema), + cache, } + .with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(2)); + + // Execution Mode + new_cache = new_cache.with_exec_mode(ExecutionMode::Bounded); + + self.cache = new_cache; + self } } @@ -403,16 +422,8 @@ impl ExecutionPlan for StatisticsExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(2) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 282b0f7079ee..8f5d7164e136 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -47,7 +47,6 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::TableReference; use datafusion_expr::{CreateExternalTable, Expr, TableType}; -use datafusion_physical_expr::PhysicalSortExpr; use async_trait::async_trait; use futures::Stream; @@ -59,6 +58,7 @@ pub use datafusion_common::test_util::{arrow_test_data, get_data_dir}; use crate::datasource::stream::{StreamConfig, StreamTable}; pub use datafusion_common::{assert_batches_eq, assert_batches_sorted_eq}; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; /// Scan an empty data source, mainly used in tests pub fn scan_empty( @@ -231,6 +231,7 @@ pub struct UnboundedExec { batch_produce: Option, batch: RecordBatch, partitions: usize, + cache: PlanPropertiesCache, } impl UnboundedExec { /// Create new exec that clones the given record batch to its output. @@ -241,11 +242,33 @@ impl UnboundedExec { batch: RecordBatch, partitions: usize, ) -> Self { + let cache = PlanPropertiesCache::new_default(batch.schema()); Self { batch_produce, batch, partitions, + cache, } + .with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + + // Output Partitioning + new_cache = new_cache + .with_partitioning(Partitioning::UnknownPartitioning(self.partitions)); + + // Execution Mode + let exec_mode = if self.batch_produce.is_none() { + ExecutionMode::Unbounded + } else { + ExecutionMode::Bounded + }; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self } } @@ -272,19 +295,8 @@ impl ExecutionPlan for UnboundedExec { self } - fn schema(&self) -> SchemaRef { - self.batch.schema() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.partitions) - } - - fn unbounded_output(&self, _children: &[bool]) -> Result { - Ok(self.batch_produce.is_none()) - } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index a9ea5cc2a35c..005b02a14054 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -30,7 +30,6 @@ use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; use datafusion::logical_expr::{ col, Expr, LogicalPlan, LogicalPlanBuilder, TableScan, UNNAMED_TABLE, }; -use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::{ collect, ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -39,25 +38,15 @@ use datafusion::scalar::ScalarValue; use datafusion_common::cast::as_primitive_array; use datafusion_common::project_schema; use datafusion_common::stats::Precision; +use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use async_trait::async_trait; -use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use futures::stream::Stream; /// Also run all tests that are found in the `custom_sources_cases` directory mod custom_sources_cases; -//--- Custom source dataframe tests ---// - -struct CustomTableProvider; -#[derive(Debug, Clone)] -struct CustomExecutionPlan { - projection: Option>, -} -struct TestCustomRecordBatchStream { - /// the nb of batches of TEST_CUSTOM_RECORD_BATCH generated - nb_batch: i32, -} macro_rules! TEST_CUSTOM_SCHEMA_REF { () => { Arc::new(Schema::new(vec![ @@ -78,6 +67,43 @@ macro_rules! TEST_CUSTOM_RECORD_BATCH { }; } +//--- Custom source dataframe tests ---// + +struct CustomTableProvider; +#[derive(Debug, Clone)] +struct CustomExecutionPlan { + projection: Option>, + cache: PlanPropertiesCache, +} + +impl CustomExecutionPlan { + fn new(projection: Option>) -> Self { + let schema = TEST_CUSTOM_SCHEMA_REF!(); + let schema = + project_schema(&schema, projection.as_ref()).expect("projected schema"); + let cache = PlanPropertiesCache::new_default(schema); + Self { projection, cache }.with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } +} + +struct TestCustomRecordBatchStream { + /// the nb of batches of TEST_CUSTOM_RECORD_BATCH generated + nb_batch: i32, +} + impl RecordBatchStream for TestCustomRecordBatchStream { fn schema(&self) -> SchemaRef { TEST_CUSTOM_SCHEMA_REF!() @@ -119,17 +145,8 @@ impl ExecutionPlan for CustomExecutionPlan { self } - fn schema(&self) -> SchemaRef { - let schema = TEST_CUSTOM_SCHEMA_REF!(); - project_schema(&schema, self.projection.as_ref()).expect("projected schema") - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -197,9 +214,7 @@ impl TableProvider for CustomTableProvider { _filters: &[Expr], _limit: Option, ) -> Result> { - Ok(Arc::new(CustomExecutionPlan { - projection: projection.cloned(), - })) + Ok(Arc::new(CustomExecutionPlan::new(projection.cloned()))) } } diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index e374abd6e891..522c18a4d654 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -25,7 +25,6 @@ use datafusion::datasource::provider::{TableProvider, TableType}; use datafusion::error::Result; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; -use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, @@ -36,6 +35,7 @@ use datafusion::scalar::ScalarValue; use datafusion_common::cast::as_primitive_array; use datafusion_common::{internal_err, not_impl_err, DataFusionError}; use datafusion_expr::expr::{BinaryExpr, Cast}; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use async_trait::async_trait; @@ -57,8 +57,28 @@ fn create_batch(value: i32, num_rows: usize) -> Result { #[derive(Debug)] struct CustomPlan { - schema: SchemaRef, batches: Vec, + cache: PlanPropertiesCache, +} + +impl CustomPlan { + fn new(schema: SchemaRef, batches: Vec) -> Self { + let cache = PlanPropertiesCache::new_default(schema); + Self { batches, cache }.with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for CustomPlan { @@ -80,16 +100,8 @@ impl ExecutionPlan for CustomPlan { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -183,25 +195,25 @@ impl TableProvider for CustomProvider { } }; - Ok(Arc::new(CustomPlan { - schema: match projection.is_empty() { + Ok(Arc::new(CustomPlan::new( + match projection.is_empty() { true => Arc::new(Schema::empty()), false => self.zero_batch.schema(), }, - batches: match int_value { + match int_value { 0 => vec![self.zero_batch.clone()], 1 => vec![self.one_batch.clone()], _ => vec![], }, - })) + ))) } - _ => Ok(Arc::new(CustomPlan { - schema: match projection.is_empty() { + _ => Ok(Arc::new(CustomPlan::new( + match projection.is_empty() { true => Arc::new(Schema::empty()), false => self.zero_batch.schema(), }, - batches: vec![], - })), + vec![], + ))), } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index f0985f554654..41915c1fd49e 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -20,21 +20,22 @@ use std::{any::Any, sync::Arc}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::execution::context::{SessionState, TaskContext}; use datafusion::{ datasource::{TableProvider, TableType}, error::Result, logical_expr::Expr, physical_plan::{ - expressions::PhysicalSortExpr, ColumnStatistics, DisplayAs, DisplayFormatType, - ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, + SendableRecordBatchStream, Statistics, }, prelude::SessionContext, scalar::ScalarValue, }; +use datafusion_common::{project_schema, stats::Precision}; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use async_trait::async_trait; -use datafusion::execution::context::{SessionState, TaskContext}; -use datafusion_common::{project_schema, stats::Precision}; /// This is a testing structure for statistics /// It will act both as a table provider and execution plan @@ -42,6 +43,7 @@ use datafusion_common::{project_schema, stats::Precision}; struct StatisticsValidation { stats: Statistics, schema: Arc, + cache: PlanPropertiesCache, } impl StatisticsValidation { @@ -51,7 +53,26 @@ impl StatisticsValidation { schema.fields().len(), "the column statistics vector length should be the number of fields" ); - Self { stats, schema } + let cache = PlanPropertiesCache::new_default(schema.clone()); + Self { + stats, + schema, + cache, + } + .with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(2)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self } } @@ -131,16 +152,8 @@ impl ExecutionPlan for StatisticsValidation { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(2) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 29708c4422ca..f6488711b9f7 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -82,16 +82,16 @@ use datafusion::{ }, optimizer::{optimize_children, OptimizerConfig, OptimizerRule}, physical_plan::{ - expressions::PhysicalSortExpr, DisplayAs, DisplayFormatType, Distribution, - ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, - Statistics, + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + RecordBatchStream, SendableRecordBatchStream, Statistics, }, physical_planner::{DefaultPhysicalPlanner, ExtensionPlanner, PhysicalPlanner}, prelude::{SessionConfig, SessionContext}, }; +use datafusion_common::arrow_datafusion_err; +use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use async_trait::async_trait; -use datafusion_common::arrow_datafusion_err; use futures::{Stream, StreamExt}; /// Execute the specified sql and return the resulting record batches @@ -395,10 +395,10 @@ impl ExtensionPlanner for TopKPlanner { assert_eq!(logical_inputs.len(), 1, "Inconsistent number of inputs"); assert_eq!(physical_inputs.len(), 1, "Inconsistent number of inputs"); // figure out input name - Some(Arc::new(TopKExec { - input: physical_inputs[0].clone(), - k: topk_node.k, - })) + Some(Arc::new(TopKExec::new( + physical_inputs[0].clone(), + topk_node.k, + ))) } else { None }, @@ -412,6 +412,27 @@ struct TopKExec { input: Arc, /// The maxium number of values k: usize, + cache: PlanPropertiesCache, +} + +impl TopKExec { + fn new(input: Arc, k: usize) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); + Self { input, k, cache }.with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl Debug for TopKExec { @@ -441,16 +462,8 @@ impl ExecutionPlan for TopKExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -465,10 +478,7 @@ impl ExecutionPlan for TopKExec { self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(TopKExec { - input: children[0].clone(), - k: self.k, - })) + Ok(Arc::new(TopKExec::new(children[0].clone(), self.k))) } /// Execute one partition and return an iterator over RecordBatch diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 156362430558..9d880e461efb 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use super::DisplayAs; +use super::{DisplayAs, ExecutionMode, PlanPropertiesCache}; use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, @@ -36,7 +36,7 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::{internal_err, not_impl_err, plan_err, DataFusionError, Result}; +use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ @@ -44,7 +44,7 @@ use datafusion_physical_expr::{ equivalence::{collapse_lex_req, ProjectionMapping}, expressions::{Column, FirstValue, LastValue, Max, Min, UnKnownColumn}, physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, - LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortRequirement, }; use itertools::Itertools; @@ -268,8 +268,7 @@ pub struct AggregateExec { required_input_ordering: Option, /// Describes how the input is ordered relative to the group by columns input_order_mode: InputOrderMode, - /// Describe how the output is ordered - output_ordering: Option, + cache: PlanPropertiesCache, } impl AggregateExec { @@ -344,7 +343,7 @@ impl AggregateExec { &new_requirement, &mut aggr_expr, &group_by, - &input_eq_properties, + input_eq_properties, &mode, )?; new_requirement.extend(req); @@ -366,11 +365,8 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - let aggregate_eqs = - input_eq_properties.project(&projection_mapping, schema.clone()); - let output_ordering = aggregate_eqs.oeq_class().output_ordering(); - - Ok(AggregateExec { + let cache = PlanPropertiesCache::new_default(schema.clone()); + let aggregate = AggregateExec { mode, group_by, aggr_expr, @@ -383,8 +379,9 @@ impl AggregateExec { required_input_ordering, limit: None, input_order_mode, - output_ordering, - }) + cache, + }; + Ok(aggregate.with_cache()) } /// Aggregation mode (full, partial) @@ -507,6 +504,55 @@ impl AggregateExec { } true } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = self + .input + .equivalence_properties() + .project(&self.projection_mapping, self.schema()); + + // Output partitioning + let mut output_partitioning = self.input.output_partitioning().clone(); + if self.mode.is_first_stage() { + // First stage aggregation will not change the output partitioning, + // but needs to respect aliases (e.g. mapping in the GROUP BY + // expression). + let input_eq_properties = self.input.equivalence_properties(); + // First stage Aggregation will not change the output partitioning but need to respect the Alias + let input_partition = self.input.output_partitioning(); + if let Partitioning::Hash(exprs, part) = input_partition { + let normalized_exprs = exprs + .iter() + .map(|expr| { + input_eq_properties + .project_expr(expr, &self.projection_mapping) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) + }) + .collect(); + output_partitioning = Partitioning::Hash(normalized_exprs, *part); + } + } + + // Unbounded Output + let mut unbounded_output = self.input.unbounded_output(); + if self.input.unbounded_output() == ExecutionMode::Unbounded + && self.input_order_mode == InputOrderMode::Linear + { + // Cannot run without breaking pipeline. + unbounded_output = ExecutionMode::InExecutable; + } + + self.cache = PlanPropertiesCache::new( + eq_properties, + output_partitioning, + unbounded_output, + ); + self + } + pub fn input_order_mode(&self) -> &InputOrderMode { &self.input_order_mode } @@ -595,58 +641,8 @@ impl ExecutionPlan for AggregateExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - let input_partition = self.input.output_partitioning(); - if self.mode.is_first_stage() { - // First stage aggregation will not change the output partitioning, - // but needs to respect aliases (e.g. mapping in the GROUP BY - // expression). - let input_eq_properties = self.input.equivalence_properties(); - // First stage Aggregation will not change the output partitioning but need to respect the Alias - let input_partition = self.input.output_partitioning(); - if let Partitioning::Hash(exprs, part) = input_partition { - let normalized_exprs = exprs - .into_iter() - .map(|expr| { - input_eq_properties - .project_expr(&expr, &self.projection_mapping) - .unwrap_or_else(|| { - Arc::new(UnKnownColumn::new(&expr.to_string())) - }) - }) - .collect(); - return Partitioning::Hash(normalized_exprs, part); - } - } - // Final Aggregation's output partitioning is the same as its real input - input_partition - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - if children[0] { - if self.input_order_mode == InputOrderMode::Linear { - // Cannot run without breaking pipeline. - plan_err!( - "Aggregate Error: `GROUP BY` clauses with columns without ordering and GROUPING SETS are not supported for unbounded inputs." - ) - } else { - Ok(true) - } - } else { - Ok(false) - } - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.output_ordering.as_deref() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -667,12 +663,6 @@ impl ExecutionPlan for AggregateExec { vec![self.required_input_ordering.clone()] } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input - .equivalence_properties() - .project(&self.projection_mapping, self.schema()) - } - fn children(&self) -> Vec> { vec![self.input.clone()] } @@ -1630,6 +1620,28 @@ mod tests { struct TestYieldingExec { /// True if this exec should yield back to runtime the first time it is polled pub yield_first: bool, + cache: PlanPropertiesCache, + } + + impl TestYieldingExec { + fn new(yield_first: bool) -> Self { + let schema = some_data().0; + let cache = PlanPropertiesCache::new_default(schema); + Self { yield_first, cache }.with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for TestYieldingExec { @@ -1650,16 +1662,9 @@ mod tests { fn as_any(&self) -> &dyn Any { self } - fn schema(&self) -> SchemaRef { - some_data().0 - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -1741,72 +1746,63 @@ mod tests { #[tokio::test] async fn aggregate_source_not_yielding() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: false }); + let input: Arc = Arc::new(TestYieldingExec::new(false)); check_aggregates(input, false).await } #[tokio::test] async fn aggregate_grouping_sets_source_not_yielding() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: false }); + let input: Arc = Arc::new(TestYieldingExec::new(false)); check_grouping_sets(input, false).await } #[tokio::test] async fn aggregate_source_with_yielding() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: true }); + let input: Arc = Arc::new(TestYieldingExec::new(true)); check_aggregates(input, false).await } #[tokio::test] async fn aggregate_grouping_sets_with_yielding() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: true }); + let input: Arc = Arc::new(TestYieldingExec::new(true)); check_grouping_sets(input, false).await } #[tokio::test] async fn aggregate_source_not_yielding_with_spill() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: false }); + let input: Arc = Arc::new(TestYieldingExec::new(false)); check_aggregates(input, true).await } #[tokio::test] async fn aggregate_grouping_sets_source_not_yielding_with_spill() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: false }); + let input: Arc = Arc::new(TestYieldingExec::new(false)); check_grouping_sets(input, true).await } #[tokio::test] async fn aggregate_source_with_yielding_with_spill() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: true }); + let input: Arc = Arc::new(TestYieldingExec::new(true)); check_aggregates(input, true).await } #[tokio::test] async fn aggregate_grouping_sets_with_yielding_with_spill() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: true }); + let input: Arc = Arc::new(TestYieldingExec::new(true)); check_grouping_sets(input, true).await } #[tokio::test] async fn test_oom() -> Result<()> { - let input: Arc = - Arc::new(TestYieldingExec { yield_first: true }); + let input: Arc = Arc::new(TestYieldingExec::new(true)); let input_schema = input.schema(); let runtime = Arc::new( diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 4f1578e220dd..7d4bdebfe8a0 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -20,9 +20,8 @@ use std::sync::Arc; use std::{any::Any, time::Instant}; -use super::expressions::PhysicalSortExpr; use super::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; -use super::{DisplayAs, Distribution, SendableRecordBatchStream}; +use super::{DisplayAs, Distribution, PlanPropertiesCache, SendableRecordBatchStream}; use crate::display::DisplayableExecutionPlan; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; @@ -45,6 +44,7 @@ pub struct AnalyzeExec { pub(crate) input: Arc, /// The output schema for RecordBatches of this exec node schema: SchemaRef, + cache: PlanPropertiesCache, } impl AnalyzeExec { @@ -55,12 +55,15 @@ impl AnalyzeExec { input: Arc, schema: SchemaRef, ) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); AnalyzeExec { verbose, show_statistics, input, schema, + cache, } + .with_cache() } /// access to verbose @@ -77,6 +80,19 @@ impl AnalyzeExec { pub fn input(&self) -> &Arc { &self.input } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for AnalyzeExec { @@ -99,8 +115,8 @@ impl ExecutionPlan for AnalyzeExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -112,26 +128,6 @@ impl ExecutionPlan for AnalyzeExec { vec![] } - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - if children[0] { - internal_err!("Streaming execution of AnalyzeExec is not possible") - } else { - Ok(false) - } - } - - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn with_new_children( self: Arc, mut children: Vec>, diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 09d1ea87ca37..8d25f7df1d8f 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -23,12 +23,10 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::{DisplayAs, Statistics}; +use super::{DisplayAs, PlanPropertiesCache, Statistics}; use crate::{ - DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, + DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; use arrow::datatypes::SchemaRef; @@ -36,7 +34,6 @@ use arrow::error::Result as ArrowResult; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::EquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -51,16 +48,20 @@ pub struct CoalesceBatchesExec { target_batch_size: usize, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl CoalesceBatchesExec { /// Create a new CoalesceBatchesExec pub fn new(input: Arc, target_batch_size: usize) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); Self { input, target_batch_size, metrics: ExecutionPlanMetricsSet::new(), + cache, } + .with_cache() } /// The input plan @@ -72,6 +73,22 @@ impl CoalesceBatchesExec { pub fn target_batch_size(&self) -> usize { self.target_batch_size } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = self.input.equivalence_properties().clone(); + + // Output Partitioning + // The coalesce batches operator does not make any changes to the partitioning of its input + let output_partitioning = self.input.output_partitioning().clone(); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for CoalesceBatchesExec { @@ -98,34 +115,14 @@ impl ExecutionPlan for CoalesceBatchesExec { self } - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - // The coalesce batches operator does not make any changes to the schema of its input - self.input.schema() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.input.clone()] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - // The coalesce batches operator does not make any changes to the partitioning of its input - self.input.output_partitioning() - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - // The coalesce batches operator does not make any changes to the sorting of its input - self.input.output_ordering() - } - fn maintains_input_order(&self) -> Vec { vec![true] } @@ -134,10 +131,6 @@ impl ExecutionPlan for CoalesceBatchesExec { vec![false] } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - fn with_new_children( self: Arc, children: Vec>, @@ -302,7 +295,7 @@ pub fn concat_batches( #[cfg(test)] mod tests { use super::*; - use crate::{memory::MemoryExec, repartition::RepartitionExec}; + use crate::{memory::MemoryExec, repartition::RepartitionExec, Partitioning}; use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::UInt32Array; diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index bfcff2853538..4815aaa2d4f0 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -21,17 +21,14 @@ use std::any::Any; use std::sync::Arc; -use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::stream::{ObservedStream, RecordBatchReceiverStream}; -use super::{DisplayAs, SendableRecordBatchStream, Statistics}; +use super::{DisplayAs, PlanPropertiesCache, SendableRecordBatchStream, Statistics}; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; -use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::EquivalenceProperties; /// Merge execution plan executes partitions in parallel and combines them into a single /// partition. No guarantees are made about the order of the resulting partition. @@ -41,21 +38,42 @@ pub struct CoalescePartitionsExec { input: Arc, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl CoalescePartitionsExec { /// Create a new CoalescePartitionsExec pub fn new(input: Arc) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); CoalescePartitionsExec { input, metrics: ExecutionPlanMetricsSet::new(), + cache, } + .with_cache() } /// Input execution plan pub fn input(&self) -> &Arc { &self.input } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let mut eq_properties = self.input.equivalence_properties().clone(); + // Coalesce partitions loses existing orderings. + eq_properties.clear_orderings(); + + // Output Partitioning + let output_partitioning = Partitioning::UnknownPartitioning(1); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for CoalescePartitionsExec { @@ -78,36 +96,14 @@ impl ExecutionPlan for CoalescePartitionsExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.input.clone()] } - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) - } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - let mut output_eq = self.input.equivalence_properties(); - // Coalesce partitions loses existing orderings. - output_eq.clear_orderings(); - output_eq - } - fn benefits_from_input_partitioning(&self) -> Vec { vec![false] } diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 19c2847b09dc..e790683a7a98 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -444,7 +444,7 @@ mod tests { use datafusion_common::DataFusionError; - use crate::{DisplayAs, ExecutionPlan}; + use crate::{DisplayAs, ExecutionPlan, PlanPropertiesCache}; use super::DisplayableExecutionPlan; @@ -470,18 +470,12 @@ mod tests { self } - fn schema(&self) -> arrow_schema::SchemaRef { - Arc::new(arrow_schema::Schema::empty()) - } - - fn output_partitioning(&self) -> datafusion_physical_expr::Partitioning { - datafusion_physical_expr::Partitioning::UnknownPartitioning(1) + fn cache(&self) -> &PlanPropertiesCache { + unimplemented!() } - fn output_ordering( - &self, - ) -> Option<&[datafusion_physical_expr::PhysicalSortExpr]> { - None + fn output_partitioning(&self) -> &datafusion_physical_expr::Partitioning { + &datafusion_physical_expr::Partitioning::UnknownPartitioning(1) } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 41c8dbed1453..dfd1e9d12904 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -20,8 +20,10 @@ use std::any::Any; use std::sync::Arc; -use super::expressions::PhysicalSortExpr; -use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; +use super::{ + common, DisplayAs, ExecutionMode, PlanPropertiesCache, SendableRecordBatchStream, + Statistics, +}; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; @@ -38,26 +40,53 @@ pub struct EmptyExec { schema: SchemaRef, /// Number of partitions partitions: usize, + cache: PlanPropertiesCache, } impl EmptyExec { /// Create a new EmptyExec pub fn new(schema: SchemaRef) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); EmptyExec { schema, partitions: 1, + cache, } + .with_cache() } /// Create a new EmptyExec with specified partition number pub fn with_partitions(mut self, partitions: usize) -> Self { self.partitions = partitions; + // Changing partitions may invalidate output partitioning. + // Update it also. + let output_partitioning = self.output_partitioning_helper(); + self.cache = self.cache.with_partitioning(output_partitioning); self } fn data(&self) -> Result> { Ok(vec![]) } + + fn output_partitioning_helper(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.partitions) + } + + fn with_cache(mut self) -> Self { + // Output Partitioning + let output_partitioning = self.output_partitioning_helper(); + + let mut new_cache = self.cache; + new_cache = new_cache.with_partitioning(output_partitioning); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for EmptyExec { @@ -80,23 +109,14 @@ impl ExecutionPlan for EmptyExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.partitions) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn with_new_children( self: Arc, _: Vec>, diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index e4904ddd3410..1f7beb632501 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -20,8 +20,7 @@ use std::any::Any; use std::sync::Arc; -use super::expressions::PhysicalSortExpr; -use super::{DisplayAs, SendableRecordBatchStream}; +use super::{DisplayAs, ExecutionMode, PlanPropertiesCache, SendableRecordBatchStream}; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; @@ -43,6 +42,7 @@ pub struct ExplainExec { stringified_plans: Vec, /// control which plans to print verbose: bool, + cache: PlanPropertiesCache, } impl ExplainExec { @@ -52,11 +52,14 @@ impl ExplainExec { stringified_plans: Vec, verbose: bool, ) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); ExplainExec { schema, stringified_plans, verbose, + cache, } + .with_cache() } /// The strings to be printed @@ -68,6 +71,19 @@ impl ExplainExec { pub fn verbose(&self) -> bool { self.verbose } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for ExplainExec { @@ -90,8 +106,8 @@ impl ExecutionPlan for ExplainExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -99,15 +115,6 @@ impl ExecutionPlan for ExplainExec { vec![] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn with_new_children( self: Arc, _: Vec>, diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 362fa10efc9f..9e0b644cd4fb 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -23,13 +23,13 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use super::expressions::PhysicalSortExpr; use super::{ - ColumnStatistics, DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, PlanPropertiesCache, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, - Column, DisplayFormatType, ExecutionPlan, Partitioning, + Column, DisplayFormatType, ExecutionPlan, }; use arrow::compute::filter_record_batch; @@ -44,8 +44,7 @@ use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ - analyze, split_conjunction, AnalysisContext, EquivalenceProperties, ExprBoundaries, - PhysicalExpr, + analyze, split_conjunction, AnalysisContext, ExprBoundaries, PhysicalExpr, }; use futures::stream::{Stream, StreamExt}; @@ -63,6 +62,7 @@ pub struct FilterExec { metrics: ExecutionPlanMetricsSet, /// Selectivity for statistics. 0 = no rows, 100 all rows default_selectivity: u8, + cache: PlanPropertiesCache, } impl FilterExec { @@ -72,12 +72,17 @@ impl FilterExec { input: Arc, ) -> Result { match predicate.data_type(input.schema().as_ref())? { - DataType::Boolean => Ok(Self { - predicate, - input: input.clone(), - metrics: ExecutionPlanMetricsSet::new(), - default_selectivity: 20, - }), + DataType::Boolean => { + let cache = PlanPropertiesCache::new_default(input.schema()); + Ok(Self { + predicate, + input: input.clone(), + metrics: ExecutionPlanMetricsSet::new(), + default_selectivity: 20, + cache, + } + .with_cache()) + } other => { plan_err!("Filter predicate must return boolean values, not {other:?}") } @@ -109,6 +114,35 @@ impl FilterExec { pub fn default_selectivity(&self) -> u8 { self.default_selectivity } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let stats = self.statistics().unwrap(); + // Combine the equal predicates with the input equivalence properties + let mut eq_properties = self.input.equivalence_properties().clone(); + let (equal_pairs, _) = collect_columns_from_predicate(&self.predicate); + for (lhs, rhs) in equal_pairs { + let lhs_expr = Arc::new(lhs.clone()) as _; + let rhs_expr = Arc::new(rhs.clone()) as _; + eq_properties.add_equal_conditions(&lhs_expr, &rhs_expr) + } + // Add the columns that have only one value (singleton) after filtering to constants. + let constants = collect_columns(self.predicate()) + .into_iter() + .filter(|column| stats.column_statistics[column.index()].is_singleton()) + .map(|column| Arc::new(column) as _); + eq_properties = eq_properties.add_constants(constants); + + // Output Partitioning + let output_partitioning = self.input.output_partitioning().clone(); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for FilterExec { @@ -131,55 +165,19 @@ impl ExecutionPlan for FilterExec { self } - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - // The filter operator does not make any changes to the schema of its input - self.input.schema() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.input.clone()] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - self.input.output_partitioning() - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input.output_ordering() - } - fn maintains_input_order(&self) -> Vec { // tell optimizer this operator doesn't reorder its input vec![true] } - fn equivalence_properties(&self) -> EquivalenceProperties { - let stats = self.statistics().unwrap(); - // Combine the equal predicates with the input equivalence properties - let mut result = self.input.equivalence_properties(); - let (equal_pairs, _) = collect_columns_from_predicate(&self.predicate); - for (lhs, rhs) in equal_pairs { - let lhs_expr = Arc::new(lhs.clone()) as _; - let rhs_expr = Arc::new(rhs.clone()) as _; - result.add_equal_conditions(&lhs_expr, &rhs_expr) - } - // Add the columns that have only one value (singleton) after filtering to constants. - let constants = collect_columns(self.predicate()) - .into_iter() - .filter(|column| stats.column_statistics[column.index()].is_singleton()) - .map(|column| Arc::new(column) as _); - result.add_constants(constants) - } - fn with_new_children( self: Arc, mut children: Vec>, diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 81cdfd753fe6..bdc1c3bc40e3 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -22,9 +22,9 @@ use std::fmt; use std::fmt::Debug; use std::sync::Arc; -use super::expressions::PhysicalSortExpr; use super::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanPropertiesCache, + SendableRecordBatchStream, }; use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; @@ -86,6 +86,7 @@ pub struct FileSinkExec { count_schema: SchemaRef, /// Optional required sort order for output data. sort_order: Option>, + cache: PlanPropertiesCache, } impl fmt::Debug for FileSinkExec { @@ -102,13 +103,17 @@ impl FileSinkExec { sink_schema: SchemaRef, sort_order: Option>, ) -> Self { + let count_schema = make_count_schema(); + let cache = PlanPropertiesCache::new_default(count_schema); Self { input, sink, sink_schema, count_schema: make_count_schema(), sort_order, + cache, } + .with_cache() } fn execute_input_stream( @@ -170,6 +175,19 @@ impl FileSinkExec { pub fn metrics(&self) -> Option { self.sink.metrics() } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for FileSinkExec { @@ -193,17 +211,8 @@ impl ExecutionPlan for FileSinkExec { self } - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - self.count_schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn benefits_from_input_partitioning(&self) -> Vec { @@ -240,17 +249,12 @@ impl ExecutionPlan for FileSinkExec { self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(Self { - input: children[0].clone(), - sink: self.sink.clone(), - sink_schema: self.sink_schema.clone(), - count_schema: self.count_schema.clone(), - sort_order: self.sort_order.clone(), - })) - } - - fn unbounded_output(&self, _children: &[bool]) -> Result { - Ok(_children[0]) + Ok(Arc::new(Self::new( + children[0].clone(), + self.sink.clone(), + self.sink_schema.clone(), + self.sort_order.clone(), + ))) } /// Execute the plan and return a stream of `RecordBatch`es for diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 938c9e4d343d..f23c78c18515 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -24,22 +24,21 @@ use super::utils::{ adjust_right_output_partitioning, BuildProbeJoinMetrics, OnceAsync, OnceFut, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::DisplayAs; use crate::{ coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, - ColumnStatistics, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayFormatType, Distribution, ExecutionPlan, + PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::{DisplayAs, ExecutionMode}; use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow_array::RecordBatchOptions; use datafusion_common::stats::Precision; -use datafusion_common::{plan_err, DataFusionError, JoinType, Result, ScalarValue}; +use datafusion_common::{JoinType, Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -61,6 +60,7 @@ pub struct CrossJoinExec { left_fut: OnceAsync, /// Execution plan metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl CrossJoinExec { @@ -76,14 +76,16 @@ impl CrossJoinExec { }; let schema = Arc::new(Schema::new(all_columns)); - + let cache = PlanPropertiesCache::new_default(schema.clone()); CrossJoinExec { left, right, schema, left_fut: Default::default(), metrics: ExecutionPlanMetricsSet::default(), + cache, } + .with_cache() } /// left (build) side which gets loaded in memory @@ -95,6 +97,41 @@ impl CrossJoinExec { pub fn right(&self) -> &Arc { &self.right } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + // TODO: Check equivalence properties of crossjoin, for some cases it may preserve ordering. + let eq_properties = join_equivalence_properties( + self.left.equivalence_properties().clone(), + self.right.equivalence_properties().clone(), + &JoinType::Full, + self.schema(), + &[false, false], + None, + &[], + ); + + // Output Partitioning + // TODO optimize CrossJoin implementation to generate M * N partitions + let left_columns_len = self.left.schema().fields.len(); + let output_partitioning = adjust_right_output_partitioning( + self.right.output_partitioning().clone(), + left_columns_len, + ); + + // Execution Mode + let left_unbounded = self.left.unbounded_output(); + let right_unbounded = self.right.unbounded_output(); + let exec_mode = match (left_unbounded, right_unbounded) { + (ExecutionMode::Bounded, ExecutionMode::Bounded) => ExecutionMode::Bounded, + // If any of the inputs is unbounded, cross join break pipeline. + (_, _) => ExecutionMode::InExecutable, + }; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } /// Asynchronously collect the result of the left child @@ -158,8 +195,8 @@ impl ExecutionPlan for CrossJoinExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -170,19 +207,6 @@ impl ExecutionPlan for CrossJoinExec { Some(self.metrics.clone_inner()) } - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - if children[0] || children[1] { - plan_err!( - "Cross Join Error: Cross join is not supported for the unbounded inputs." - ) - } else { - Ok(false) - } - } - fn with_new_children( self: Arc, children: Vec>, @@ -200,32 +224,6 @@ impl ExecutionPlan for CrossJoinExec { ] } - // TODO optimize CrossJoin implementation to generate M * N partitions - fn output_partitioning(&self) -> Partitioning { - let left_columns_len = self.left.schema().fields.len(); - adjust_right_output_partitioning( - self.right.output_partitioning(), - left_columns_len, - ) - } - - // TODO check the output ordering of CrossJoin - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - join_equivalence_properties( - self.left.equivalence_properties(), - self.right.equivalence_properties(), - &JoinType::Full, - self.schema(), - &[false, false], - None, - &[], - ) - } - fn execute( &self, partition: usize, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index cd8b17d13598..aee207dc2871 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -25,12 +25,12 @@ use std::{any::Any, usize, vec}; use crate::joins::utils::{ adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices, - calculate_join_output_ordering, get_final_indices_from_bit_map, - need_produce_result_in_final, JoinHashMap, JoinHashMapOffset, JoinHashMapType, + get_final_indices_from_bit_map, need_produce_result_in_final, JoinHashMap, + JoinHashMapOffset, JoinHashMapType, }; use crate::{ coalesce_partitions::CoalescePartitionsExec, - expressions::PhysicalSortExpr, + exec_mode_flatten, hash_utils::create_hashes, joins::utils::{ adjust_right_output_partitioning, build_join_schema, check_join_is_valid, @@ -38,8 +38,8 @@ use crate::{ BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, StatefulStreamResult, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::{handle_state, DisplayAs}; @@ -66,7 +66,7 @@ use datafusion_common::{ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalExprRef}; +use datafusion_physical_expr::PhysicalExprRef; use ahash::RandomState; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -288,8 +288,6 @@ pub struct HashJoinExec { left_fut: OnceAsync, /// Shared the `RandomState` for the hashing algorithm random_state: RandomState, - /// Output order - output_order: Option>, /// Partitioning mode to use pub mode: PartitionMode, /// Execution metrics @@ -301,6 +299,7 @@ pub struct HashJoinExec { /// Otherwise, rows that have `null`s in the join columns will not be /// matched and thus will not appear in the output. pub null_equals_null: bool, + cache: PlanPropertiesCache, } impl HashJoinExec { @@ -330,15 +329,7 @@ impl HashJoinExec { let random_state = RandomState::with_seeds(0, 0, 0, 0); - let output_order = calculate_join_output_ordering( - left.output_ordering().unwrap_or(&[]), - right.output_ordering().unwrap_or(&[]), - *join_type, - &on, - left_schema.fields.len(), - &Self::maintains_input_order(*join_type), - Some(Self::probe_side()), - ); + let cache = PlanPropertiesCache::new_default(Arc::new(schema.clone())); Ok(HashJoinExec { left, @@ -353,8 +344,9 @@ impl HashJoinExec { metrics: ExecutionPlanMetricsSet::new(), column_indices, null_equals_null, - output_order, - }) + cache, + } + .with_cache()) } /// left (build) side which gets hashed @@ -408,6 +400,76 @@ impl HashJoinExec { // In current implementation right side is always probe side. JoinSide::Right } + + fn with_cache(mut self) -> Self { + let left = &self.left; + let right = &self.right; + let schema = self.schema(); + // Equivalence properties + let eq_properties = join_equivalence_properties( + left.equivalence_properties().clone(), + right.equivalence_properties().clone(), + &self.join_type, + schema, + &Self::maintains_input_order(self.join_type), + Some(Self::probe_side()), + &self.on, + ); + + // Output partitioning + let left_columns_len = left.schema().fields.len(); + let output_partitioning = match self.mode { + PartitionMode::CollectLeft => match self.join_type { + JoinType::Inner | JoinType::Right => adjust_right_output_partitioning( + right.output_partitioning().clone(), + left_columns_len, + ), + JoinType::RightSemi | JoinType::RightAnti => { + right.output_partitioning().clone() + } + JoinType::Left + | JoinType::LeftSemi + | JoinType::LeftAnti + | JoinType::Full => Partitioning::UnknownPartitioning( + right.output_partitioning().partition_count(), + ), + }, + PartitionMode::Partitioned => partitioned_join_output_partitioning( + self.join_type, + left.output_partitioning().clone(), + right.output_partitioning().clone(), + left_columns_len, + ), + PartitionMode::Auto => Partitioning::UnknownPartitioning( + right.output_partitioning().partition_count(), + ), + }; + + // Unbounded output + let left_unbounded = left.unbounded_output().is_unbounded(); + let right_unbounded = right.unbounded_output().is_unbounded(); + // If left is unbounded, or right is unbounded with JoinType::Right, + // JoinType::Full, JoinType::RightAnti types. + let breaking = left_unbounded + || (right_unbounded + && matches!( + self.join_type, + JoinType::Left + | JoinType::Full + | JoinType::LeftAnti + | JoinType::LeftSemi + )); + + let exec_mode = if breaking { + ExecutionMode::InExecutable + } else { + exec_mode_flatten([left, right]) + }; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for HashJoinExec { @@ -439,8 +501,8 @@ impl ExecutionPlan for HashJoinExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -464,71 +526,6 @@ impl ExecutionPlan for HashJoinExec { } } - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - let (left, right) = (children[0], children[1]); - // If left is unbounded, or right is unbounded with JoinType::Right, - // JoinType::Full, JoinType::RightAnti types. - let breaking = left - || (right - && matches!( - self.join_type, - JoinType::Left - | JoinType::Full - | JoinType::LeftAnti - | JoinType::LeftSemi - )); - - if breaking { - plan_err!( - "Join Error: The join with cannot be executed with unbounded inputs. {}", - if left && right { - "Currently, we do not support unbounded inputs on both sides." - } else { - "Please consider a different type of join or sources." - } - ) - } else { - Ok(left || right) - } - } - - fn output_partitioning(&self) -> Partitioning { - let left_columns_len = self.left.schema().fields.len(); - match self.mode { - PartitionMode::CollectLeft => match self.join_type { - JoinType::Inner | JoinType::Right => adjust_right_output_partitioning( - self.right.output_partitioning(), - left_columns_len, - ), - JoinType::RightSemi | JoinType::RightAnti => { - self.right.output_partitioning() - } - JoinType::Left - | JoinType::LeftSemi - | JoinType::LeftAnti - | JoinType::Full => Partitioning::UnknownPartitioning( - self.right.output_partitioning().partition_count(), - ), - }, - PartitionMode::Partitioned => partitioned_join_output_partitioning( - self.join_type, - self.left.output_partitioning(), - self.right.output_partitioning(), - left_columns_len, - ), - PartitionMode::Auto => Partitioning::UnknownPartitioning( - self.right.output_partitioning().partition_count(), - ), - } - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.output_order.as_deref() - } - // For [JoinType::Inner] and [JoinType::RightSemi] in hash joins, the probe phase initiates by // applying the hash function to convert the join key(s) in each row into a hash value from the // probe side table in the order they're arranged. The hash value is used to look up corresponding @@ -549,18 +546,6 @@ impl ExecutionPlan for HashJoinExec { Self::maintains_input_order(self.join_type) } - fn equivalence_properties(&self) -> EquivalenceProperties { - join_equivalence_properties( - self.left.equivalence_properties(), - self.right.equivalence_properties(), - &self.join_type, - self.schema(), - &self.maintains_input_order(), - Some(Self::probe_side()), - self.on(), - ) - } - fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index f89a2445fd07..724735f28077 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -34,8 +34,8 @@ use crate::joins::utils::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, + exec_mode_safe_flatten, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, + PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, }; use arrow::array::{ @@ -49,7 +49,6 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_expr::JoinType; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -93,6 +92,7 @@ pub struct NestedLoopJoinExec { column_indices: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl NestedLoopJoinExec { @@ -108,6 +108,7 @@ impl NestedLoopJoinExec { check_join_is_valid(&left_schema, &right_schema, &[])?; let (schema, column_indices) = build_join_schema(&left_schema, &right_schema, join_type); + let cache = PlanPropertiesCache::new_default(Arc::new(schema.clone())); Ok(NestedLoopJoinExec { left, right, @@ -117,7 +118,9 @@ impl NestedLoopJoinExec { inner_table: Default::default(), column_indices, metrics: Default::default(), - }) + cache, + } + .with_cache()) } /// left side @@ -139,6 +142,40 @@ impl NestedLoopJoinExec { pub fn join_type(&self) -> &JoinType { &self.join_type } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = join_equivalence_properties( + self.left.equivalence_properties().clone(), + self.right.equivalence_properties().clone(), + &self.join_type, + self.schema(), + &self.maintains_input_order(), + None, + // No on columns in nested loop join + &[], + ); + + // Output Partitioning + // the partition of output is determined by the rule of `required_input_distribution` + let output_partitioning = if self.join_type == JoinType::Full { + self.left.output_partitioning().clone() + } else { + partitioned_join_output_partitioning( + self.join_type, + self.left.output_partitioning().clone(), + self.right.output_partitioning().clone(), + self.left.schema().fields.len(), + ) + }; + + // Execution Mode + let exec_mode = exec_mode_safe_flatten([&self.left, &self.right]); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for NestedLoopJoinExec { @@ -164,46 +201,14 @@ impl ExecutionPlan for NestedLoopJoinExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - // the partition of output is determined by the rule of `required_input_distribution` - if self.join_type == JoinType::Full { - self.left.output_partitioning() - } else { - partitioned_join_output_partitioning( - self.join_type, - self.left.output_partitioning(), - self.right.output_partitioning(), - self.left.schema().fields.len(), - ) - } - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - // no specified order for the output - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { distribution_from_join_type(&self.join_type) } - fn equivalence_properties(&self) -> EquivalenceProperties { - join_equivalence_properties( - self.left.equivalence_properties(), - self.right.equivalence_properties(), - &self.join_type, - self.schema(), - &self.maintains_input_order(), - None, - // No on columns in nested loop join - &[], - ) - } - fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] } @@ -754,7 +759,7 @@ mod tests { use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; - use datafusion_physical_expr::PhysicalExpr; + use datafusion_physical_expr::{Partitioning, PhysicalExpr}; fn build_table( a: (&str, &Vec), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 7af614e53491..2f9729351a31 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -32,13 +32,14 @@ use std::task::{Context, Poll}; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ - build_join_schema, calculate_join_output_ordering, check_join_is_valid, - estimate_join_statistics, partitioned_join_output_partitioning, JoinFilter, JoinOn, + build_join_schema, check_join_is_valid, estimate_join_statistics, + partitioned_join_output_partitioning, JoinFilter, JoinOn, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ - metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + exec_mode_flatten, metrics, DisplayAs, DisplayFormatType, Distribution, + ExecutionPlan, PhysicalExpr, PlanPropertiesCache, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; use arrow::array::*; @@ -53,9 +54,7 @@ use datafusion_common::{ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; -use datafusion_physical_expr::{ - EquivalenceProperties, PhysicalExprRef, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use futures::{Stream, StreamExt}; @@ -81,12 +80,11 @@ pub struct SortMergeJoinExec { left_sort_exprs: Vec, /// The right SortExpr right_sort_exprs: Vec, - /// The output ordering - output_ordering: Option>, /// Sort options of join columns used in sorting left and right execution plans pub sort_options: Vec, /// If null_equals_null is true, null == null else null != null pub null_equals_null: bool, + cache: PlanPropertiesCache, } impl SortMergeJoinExec { @@ -137,19 +135,9 @@ impl SortMergeJoinExec { }) .unzip(); - let output_ordering = calculate_join_output_ordering( - left.output_ordering().unwrap_or(&[]), - right.output_ordering().unwrap_or(&[]), - join_type, - &on, - left_schema.fields.len(), - &Self::maintains_input_order(join_type), - Some(Self::probe_side(&join_type)), - ); - let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); - + let cache = PlanPropertiesCache::new_default(schema.clone()); Ok(Self { left, right, @@ -160,10 +148,11 @@ impl SortMergeJoinExec { metrics: ExecutionPlanMetricsSet::new(), left_sort_exprs, right_sort_exprs, - output_ordering, sort_options, null_equals_null, - }) + cache, + } + .with_cache()) } /// Get probe side (e.g streaming side) information for this sort merge join. @@ -211,6 +200,35 @@ impl SortMergeJoinExec { pub fn left(&self) -> &dyn ExecutionPlan { self.left.as_ref() } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = join_equivalence_properties( + self.left.equivalence_properties().clone(), + self.right.equivalence_properties().clone(), + &self.join_type, + self.schema(), + &self.maintains_input_order(), + Some(Self::probe_side(&self.join_type)), + self.on(), + ); + + // Output Partitioning + let left_columns_len = self.left.schema().fields.len(); + let output_partitioning = partitioned_join_output_partitioning( + self.join_type, + self.left.output_partitioning().clone(), + self.right.output_partitioning().clone(), + left_columns_len, + ); + + // Execution Mode + let exec_mode = exec_mode_flatten([&self.left, &self.right]); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for SortMergeJoinExec { @@ -243,8 +261,8 @@ impl ExecutionPlan for SortMergeJoinExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -267,36 +285,10 @@ impl ExecutionPlan for SortMergeJoinExec { ] } - fn output_partitioning(&self) -> Partitioning { - let left_columns_len = self.left.schema().fields.len(); - partitioned_join_output_partitioning( - self.join_type, - self.left.output_partitioning(), - self.right.output_partitioning(), - left_columns_len, - ) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.output_ordering.as_deref() - } - fn maintains_input_order(&self) -> Vec { Self::maintains_input_order(self.join_type) } - fn equivalence_properties(&self) -> EquivalenceProperties { - join_equivalence_properties( - self.left.equivalence_properties(), - self.right.equivalence_properties(), - &self.join_type, - self.schema(), - &self.maintains_input_order(), - Some(Self::probe_side(&self.join_type)), - self.on(), - ) - } - fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 42c7029edcc1..f636fc197168 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -46,11 +46,12 @@ use crate::joins::utils::{ JoinHashMapType, JoinOn, StatefulStreamResult, }; use crate::{ + exec_mode_flatten, expressions::PhysicalSortExpr, joins::StreamJoinPartitionMode, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, - Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, PlanPropertiesCache, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::{ @@ -176,8 +177,6 @@ pub struct SymmetricHashJoinExec { pub(crate) filter: Option, /// How the join is performed pub(crate) join_type: JoinType, - /// The schema once the join is applied - schema: SchemaRef, /// Shares the `RandomState` for the hashing algorithm random_state: RandomState, /// Execution metrics @@ -192,6 +191,7 @@ pub struct SymmetricHashJoinExec { pub(crate) right_sort_exprs: Option>, /// Partition Mode mode: StreamJoinPartitionMode, + cache: PlanPropertiesCache, } impl SymmetricHashJoinExec { @@ -233,13 +233,13 @@ impl SymmetricHashJoinExec { // Initialize the random state for the join operation: let random_state = RandomState::with_seeds(0, 0, 0, 0); + let cache = PlanPropertiesCache::new_default(Arc::new(schema)); Ok(SymmetricHashJoinExec { left, right, on, filter, join_type: *join_type, - schema: Arc::new(schema), random_state, metrics: ExecutionPlanMetricsSet::new(), column_indices, @@ -247,7 +247,39 @@ impl SymmetricHashJoinExec { left_sort_exprs, right_sort_exprs, mode, - }) + cache, + } + .with_cache()) + } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = join_equivalence_properties( + self.left.equivalence_properties().clone(), + self.right.equivalence_properties().clone(), + &self.join_type, + self.schema(), + &self.maintains_input_order(), + // Has alternating probe side + None, + self.on(), + ); + + // Output Partitioning + let left_columns_len = self.left.schema().fields.len(); + let output_partitioning = partitioned_join_output_partitioning( + self.join_type, + self.left.output_partitioning().clone(), + self.right.output_partitioning().clone(), + left_columns_len, + ); + + // Execution Mode + let exec_mode = exec_mode_flatten([&self.left, &self.right]); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self } /// left stream @@ -353,12 +385,8 @@ impl ExecutionPlan for SymmetricHashJoinExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children.iter().any(|u| *u)) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -391,34 +419,6 @@ impl ExecutionPlan for SymmetricHashJoinExec { ] } - fn output_partitioning(&self) -> Partitioning { - let left_columns_len = self.left.schema().fields.len(); - partitioned_join_output_partitioning( - self.join_type, - self.left.output_partitioning(), - self.right.output_partitioning(), - left_columns_len, - ) - } - - // TODO: Output ordering might be kept for some cases. - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - join_equivalence_properties( - self.left.equivalence_properties(), - self.right.equivalence_properties(), - &self.join_type, - self.schema(), - &self.maintains_input_order(), - // Has alternating probe side - None, - self.on(), - ) - } - fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 1c4a6ac0ecaf..6805b8448df8 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -31,11 +31,11 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::tree_node::Transformed; use datafusion_common::utils::DataPtr; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ - EquivalenceProperties, PhysicalSortExpr, PhysicalSortRequirement, + EquivalenceProperties, LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, }; use futures::stream::TryStreamExt; @@ -121,21 +121,23 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { fn as_any(&self) -> &dyn Any; /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef; + fn schema(&self) -> SchemaRef { + self.cache().schema().clone() + } + + fn cache(&self) -> &PlanPropertiesCache; /// Specifies how the output of this `ExecutionPlan` is split into /// partitions. - fn output_partitioning(&self) -> Partitioning; + fn output_partitioning(&self) -> &Partitioning { + &self.cache().partitioning + } /// Specifies whether this plan generates an infinite stream of records. /// If the plan does not support pipelining, but its input(s) are /// infinite, returns an error to indicate this. - fn unbounded_output(&self, _children: &[bool]) -> Result { - if _children.iter().any(|&x| x) { - plan_err!("Plan does not support infinite stream from its children") - } else { - Ok(false) - } + fn unbounded_output(&self) -> ExecutionMode { + self.cache().exec_mode } /// If the output of this `ExecutionPlan` within each partition is sorted, @@ -148,7 +150,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// /// It is safe to return `None` here if your `ExecutionPlan` does not /// have any particular output order here - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]>; + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + self.cache().output_ordering.as_deref() + } /// Specifies the data distribution requirements for all the /// children for this `ExecutionPlan`, By default it's [[Distribution::UnspecifiedDistribution]] for each child, @@ -225,8 +229,8 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// /// See also [`Self::maintains_input_order`] and [`Self::output_ordering`] /// for related concepts. - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new(self.schema()) + fn equivalence_properties(&self) -> &EquivalenceProperties { + &self.cache().eq_properties } /// Get a list of children `ExecutionPlan`s that act as inputs to this plan. @@ -446,6 +450,147 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } } +#[derive(Clone, Copy, PartialEq, Debug)] +pub enum ExecutionMode { + Bounded, + Unbounded, + InExecutable, +} + +impl ExecutionMode { + pub fn is_unbounded(&self) -> bool { + matches!(self, ExecutionMode::Unbounded) + } + + pub fn is_executable(&self) -> bool { + matches!(self, ExecutionMode::Bounded | ExecutionMode::Unbounded) + } +} + +fn exec_mode_flatten_helper(modes: &[ExecutionMode]) -> ExecutionMode { + let mut result = ExecutionMode::Bounded; + for mode in modes { + match (mode, result) { + (ExecutionMode::InExecutable, _) | (_, ExecutionMode::InExecutable) => { + // If any of the modes is `InExecutable`. result is `InExecutable` + // Return immediately + return ExecutionMode::InExecutable; + } + ( + ExecutionMode::Unbounded, + ExecutionMode::Bounded | ExecutionMode::Unbounded, + ) + | (ExecutionMode::Bounded, ExecutionMode::Unbounded) => { + // Unbounded mode eats up bounded mode. + result = ExecutionMode::Unbounded; + } + (ExecutionMode::Bounded, ExecutionMode::Bounded) => { + // When both modes are bounded, result is bounded + result = ExecutionMode::Bounded; + } + } + } + result +} + +fn exec_mode_flatten<'a>( + children: impl IntoIterator>, +) -> ExecutionMode { + let modes = children + .into_iter() + .map(|child| child.unbounded_output()) + .collect::>(); + exec_mode_flatten_helper(&modes) +} + +fn exec_mode_safe_flatten_helper(modes: &[ExecutionMode]) -> ExecutionMode { + let mut result = ExecutionMode::Bounded; + for mode in modes { + match (mode, result) { + (ExecutionMode::Unbounded | ExecutionMode::InExecutable, _) + | (_, ExecutionMode::Unbounded | ExecutionMode::InExecutable) => { + // If any of the modes is `InExecutable`. result is `InExecutable` + // Return immediately + return ExecutionMode::InExecutable; + } + (ExecutionMode::Bounded, ExecutionMode::Bounded) => { + // When both modes are bounded, result is bounded + result = ExecutionMode::Bounded; + } + } + } + result +} + +fn exec_mode_safe_flatten<'a>( + children: impl IntoIterator>, +) -> ExecutionMode { + let modes = children + .into_iter() + .map(|child| child.unbounded_output()) + .collect::>(); + exec_mode_safe_flatten_helper(&modes) +} + +#[derive(Debug, Clone)] +pub struct PlanPropertiesCache { + pub eq_properties: EquivalenceProperties, + pub partitioning: Partitioning, + pub exec_mode: ExecutionMode, + output_ordering: Option, +} + +impl PlanPropertiesCache { + pub fn new( + eq_properties: EquivalenceProperties, + partitioning: Partitioning, + exec_mode: ExecutionMode, + ) -> Self { + let output_ordering = eq_properties.oeq_class().output_ordering(); + Self { + eq_properties, + partitioning, + exec_mode, + output_ordering, + } + } + + pub fn new_default(schema: SchemaRef) -> PlanPropertiesCache { + // Defaults are most restrictive possible values. + let eq_properties = EquivalenceProperties::new(schema); + let partitioning = Partitioning::UnknownPartitioning(0); + let exec_mode = ExecutionMode::InExecutable; + let output_ordering = None; + Self { + eq_properties, + partitioning, + exec_mode, + output_ordering, + } + } + + pub fn with_partitioning(mut self, partitioning: Partitioning) -> Self { + self.partitioning = partitioning; + self + } + + pub fn with_exec_mode(mut self, exec_mode: ExecutionMode) -> Self { + self.exec_mode = exec_mode; + self + } + + pub fn with_eq_properties(mut self, eq_properties: EquivalenceProperties) -> Self { + self.output_ordering = eq_properties.oeq_class().output_ordering(); + self.eq_properties = eq_properties; + self + } + + /// Get schema of the node. + fn schema(&self) -> &SchemaRef { + self.eq_properties.schema() + } +} + /// Indicate whether a data exchange is needed for the input of `plan`, which will be very helpful /// especially for the distributed engine to judge whether need to deal with shuffling. /// Currently there are 3 kinds of execution plan which needs data exchange @@ -595,17 +740,6 @@ pub fn execute_stream_partitioned( Ok(streams) } -// Get output (un)boundedness information for the given `plan`. -pub fn unbounded_output(plan: &Arc) -> bool { - let children_unbounded_output = plan - .children() - .iter() - .map(unbounded_output) - .collect::>(); - plan.unbounded_output(&children_unbounded_output) - .unwrap_or(true) -} - /// Utility function yielding a string representation of the given [`ExecutionPlan`]. pub fn get_plan_string(plan: &Arc) -> Vec { let formatted = displayable(plan.as_ref()).indent(true).to_string(); diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index c31d5f62c726..8b7313436469 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -22,12 +22,12 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; -use crate::{ - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, +use super::{ + DisplayAs, ExecutionMode, PlanPropertiesCache, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; +use crate::{DisplayFormatType, Distribution, ExecutionPlan, Partitioning}; use arrow::array::ArrayRef; use arrow::datatypes::SchemaRef; @@ -51,17 +51,21 @@ pub struct GlobalLimitExec { fetch: Option, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl GlobalLimitExec { /// Create a new GlobalLimitExec pub fn new(input: Arc, skip: usize, fetch: Option) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); GlobalLimitExec { input, skip, fetch, metrics: ExecutionPlanMetricsSet::new(), + cache, } + .with_cache() } /// Input execution plan @@ -78,6 +82,21 @@ impl GlobalLimitExec { pub fn fetch(&self) -> Option { self.fetch } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = self.input.equivalence_properties().clone(); + + // Output Partitioning + let output_partitioning = Partitioning::UnknownPartitioning(1); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for GlobalLimitExec { @@ -105,8 +124,8 @@ impl ExecutionPlan for GlobalLimitExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -116,10 +135,6 @@ impl ExecutionPlan for GlobalLimitExec { fn required_input_distribution(&self) -> Vec { vec![Distribution::SinglePartition] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } fn maintains_input_order(&self) -> Vec { vec![true] @@ -129,14 +144,6 @@ impl ExecutionPlan for GlobalLimitExec { vec![false] } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input.output_ordering() - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - fn with_new_children( self: Arc, children: Vec>, @@ -148,10 +155,6 @@ impl ExecutionPlan for GlobalLimitExec { ))) } - fn unbounded_output(&self, _children: &[bool]) -> Result { - Ok(false) - } - fn execute( &self, partition: usize, @@ -272,16 +275,20 @@ pub struct LocalLimitExec { fetch: usize, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl LocalLimitExec { /// Create a new LocalLimitExec partition pub fn new(input: Arc, fetch: usize) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); Self { input, fetch, metrics: ExecutionPlanMetricsSet::new(), + cache, } + .with_cache() } /// Input execution plan @@ -293,6 +300,21 @@ impl LocalLimitExec { pub fn fetch(&self) -> usize { self.fetch } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = self.input.equivalence_properties().clone(); + + // Output Partitioning + let output_partitioning = self.input.output_partitioning().clone(); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for LocalLimitExec { @@ -315,39 +337,22 @@ impl ExecutionPlan for LocalLimitExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.input.clone()] } - fn output_partitioning(&self) -> Partitioning { - self.input.output_partitioning() - } - fn benefits_from_input_partitioning(&self) -> Vec { vec![false] } - // Local limit will not change the input plan's ordering - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input.output_ordering() - } - fn maintains_input_order(&self) -> Vec { vec![true] } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - - fn unbounded_output(&self, _children: &[bool]) -> Result { - Ok(false) - } - fn with_new_children( self: Arc, children: Vec>, diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 7de474fda11c..3a994f41a0be 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -24,8 +24,8 @@ use std::task::{Context, Poll}; use super::expressions::PhysicalSortExpr; use super::{ - common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + common, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::datatypes::SchemaRef; @@ -48,6 +48,7 @@ pub struct MemoryExec { projection: Option>, // Sort information: one or more equivalent orderings sort_information: Vec, + cache: PlanPropertiesCache, } impl fmt::Debug for MemoryExec { @@ -100,9 +101,8 @@ impl ExecutionPlan for MemoryExec { self } - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -110,21 +110,6 @@ impl ExecutionPlan for MemoryExec { vec![] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.partitions.len()) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.sort_information - .first() - .map(|ordering| ordering.as_slice()) - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new_with_orderings(self.schema(), &self.sort_information) - } - fn with_new_children( self: Arc, children: Vec>, @@ -168,13 +153,16 @@ impl MemoryExec { projection: Option>, ) -> Result { let projected_schema = project_schema(&schema, projection.as_ref())?; + let cache = PlanPropertiesCache::new_default(projected_schema.clone()); Ok(Self { partitions: partitions.to_vec(), schema, projected_schema, projection, sort_information: vec![], - }) + cache, + } + .with_cache()) } pub fn partitions(&self) -> &[Vec] { @@ -203,12 +191,36 @@ impl MemoryExec { /// and treat `a ASC` and `b DESC` as the same ordering requirement. pub fn with_sort_information(mut self, sort_information: Vec) -> Self { self.sort_information = sort_information; - self + + // With updated sort information, we need to update equivalence properties also. + let eq_properties = self.equivalent_properties_helper(); + self.cache = self.cache.with_eq_properties(eq_properties); + self.with_cache() } pub fn original_schema(&self) -> SchemaRef { self.schema.clone() } + + fn equivalent_properties_helper(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings(self.schema(), &self.sort_information) + } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = self.equivalent_properties_helper(); + + // Output Partitioning + let output_partitioning = + Partitioning::UnknownPartitioning(self.partitions.len()); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } /// Iterator over batches @@ -292,7 +304,7 @@ mod tests { Field::new("b", DataType::Int64, false), Field::new("c", DataType::Int64, false), ])); - let expected_output_order = vec![ + let sort1 = vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), @@ -302,18 +314,22 @@ mod tests { options: SortOptions::default(), }, ]; - let expected_order_eq = vec![PhysicalSortExpr { + let sort2 = vec![PhysicalSortExpr { expr: col("c", &schema)?, options: SortOptions::default(), }]; - let sort_information = - vec![expected_output_order.clone(), expected_order_eq.clone()]; + let mut expected_output_order = vec![]; + expected_output_order.extend(sort1.clone()); + expected_output_order.extend(sort2.clone()); + + let sort_information = vec![sort1.clone(), sort2.clone()]; let mem_exec = MemoryExec::try_new(&[vec![]], schema, None)? .with_sort_information(sort_information); assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); let eq_properties = mem_exec.equivalence_properties(); - assert!(eq_properties.oeq_class().contains(&expected_order_eq)); + assert!(eq_properties.oeq_class().contains(&sort1)); + assert!(eq_properties.oeq_class().contains(&sort2)); Ok(()) } } diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 3ab3de62f37a..a35d259f153a 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -20,8 +20,10 @@ use std::any::Any; use std::sync::Arc; -use super::expressions::PhysicalSortExpr; -use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; +use super::{ + common, DisplayAs, ExecutionMode, PlanPropertiesCache, SendableRecordBatchStream, + Statistics, +}; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::array::{ArrayRef, NullArray}; @@ -40,20 +42,27 @@ pub struct PlaceholderRowExec { schema: SchemaRef, /// Number of partitions partitions: usize, + cache: PlanPropertiesCache, } impl PlaceholderRowExec { /// Create a new PlaceholderRowExec pub fn new(schema: SchemaRef) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); PlaceholderRowExec { schema, partitions: 1, + cache, } + .with_cache() } /// Create a new PlaceholderRowExecPlaceholderRowExec with specified partition number pub fn with_partitions(mut self, partitions: usize) -> Self { self.partitions = partitions; + // When changing partitions, output partitions should change also. + let output_partitioning = self.output_partitioning_helper(); + self.cache = self.cache.with_partitioning(output_partitioning); self } @@ -79,6 +88,24 @@ impl PlaceholderRowExec { )?] }) } + + fn output_partitioning_helper(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.partitions) + } + + fn with_cache(mut self) -> Self { + let output_partitioning = self.output_partitioning_helper(); + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(output_partitioning); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for PlaceholderRowExec { @@ -101,23 +128,14 @@ impl ExecutionPlan for PlaceholderRowExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.partitions) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn with_new_children( self: Arc, _: Vec>, diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index cc2ab62049ed..eb45e701544e 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -26,9 +26,12 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use super::expressions::{Column, PhysicalSortExpr}; +use super::expressions::Column; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; +use super::{ + DisplayAs, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + Statistics, +}; use crate::{ ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; @@ -40,7 +43,6 @@ use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; -use datafusion_physical_expr::EquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -54,13 +56,12 @@ pub struct ProjectionExec { schema: SchemaRef, /// The input plan input: Arc, - /// The output ordering - output_ordering: Option>, /// The mapping used to normalize expressions like Partitioning and /// PhysicalSortExpr that maps input to output projection_mapping: ProjectionMapping, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl ProjectionExec { @@ -94,19 +95,16 @@ impl ProjectionExec { // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; - - let input_eqs = input.equivalence_properties(); - let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); - let output_ordering = project_eqs.oeq_class().output_ordering(); - - Ok(Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); + let projection = Self { expr, schema, input, - output_ordering, projection_mapping, metrics: ExecutionPlanMetricsSet::new(), - }) + cache, + }; + Ok(projection.with_cache()) } /// The projection expressions stored as tuples of (expression, output column name) @@ -118,6 +116,46 @@ impl ProjectionExec { pub fn input(&self) -> &Arc { &self.input } + + fn with_cache(mut self) -> Self { + let input = &self.input; + // Equivalence properties + let input_eq_properties = input.equivalence_properties(); + let eq_properties = + input_eq_properties.project(&self.projection_mapping, self.schema.clone()); + + // output partitioning + // Output partition need to respect the alias + let input_partition = input.output_partitioning(); + let output_partitioning = if let Partitioning::Hash(exprs, part) = input_partition + { + let normalized_exprs = exprs + .iter() + .map(|expr| { + input_eq_properties + .project_expr(expr, &self.projection_mapping) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) + }) + .collect(); + Partitioning::Hash(normalized_exprs, *part) + } else { + input_partition.clone() + }; + + // unbounded output + let unbounded_output = input.unbounded_output(); + + // Construct cache + let cache = PlanPropertiesCache::new( + eq_properties, + output_partitioning, + unbounded_output, + ); + self.cache = cache; + self + } } impl DisplayAs for ProjectionExec { @@ -153,59 +191,19 @@ impl ExecutionPlan for ProjectionExec { self } - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.input.clone()] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - // Output partition need to respect the alias - let input_partition = self.input.output_partitioning(); - let input_eq_properties = self.input.equivalence_properties(); - if let Partitioning::Hash(exprs, part) = input_partition { - let normalized_exprs = exprs - .into_iter() - .map(|expr| { - input_eq_properties - .project_expr(&expr, &self.projection_mapping) - .unwrap_or_else(|| { - Arc::new(UnKnownColumn::new(&expr.to_string())) - }) - }) - .collect(); - Partitioning::Hash(normalized_exprs, part) - } else { - input_partition - } - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.output_ordering.as_deref() - } - fn maintains_input_order(&self) -> Vec { // tell optimizer this operator doesn't reorder its input vec![true] } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input - .equivalence_properties() - .project(&self.projection_mapping, self.schema()) - } - fn with_new_children( self: Arc, mut children: Vec>, diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index 614ab990ac49..e49c94d6c646 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -21,14 +21,13 @@ use std::any::Any; use std::sync::Arc; use std::task::{Context, Poll}; -use super::expressions::PhysicalSortExpr; use super::metrics::BaselineMetrics; -use super::RecordBatchStream; use super::{ metrics::{ExecutionPlanMetricsSet, MetricsSet}, work_table::{WorkTable, WorkTableExec}, - SendableRecordBatchStream, Statistics, + ExecutionMode, SendableRecordBatchStream, Statistics, }; +use super::{PlanPropertiesCache, RecordBatchStream}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -69,6 +68,7 @@ pub struct RecursiveQueryExec { is_distinct: bool, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl RecursiveQueryExec { @@ -83,6 +83,7 @@ impl RecursiveQueryExec { let work_table = Arc::new(WorkTable::new()); // Use the same work table for both the WorkTableExec and the recursive term let recursive_term = assign_work_table(recursive_term, work_table.clone())?; + let cache = PlanPropertiesCache::new_default(static_term.schema()); Ok(RecursiveQueryExec { name, static_term, @@ -90,7 +91,22 @@ impl RecursiveQueryExec { is_distinct, work_table, metrics: ExecutionPlanMetricsSet::new(), - }) + cache, + } + .with_cache()) + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self } } @@ -99,21 +115,14 @@ impl ExecutionPlan for RecursiveQueryExec { self } - fn schema(&self) -> SchemaRef { - self.static_term.schema() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.static_term.clone(), self.recursive_term.clone()] } - // Distribution on a recursive query is really tricky to handle. - // For now, we are going to use a single partition but in the - // future we might find a better way to handle this. - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - // TODO: control these hints and see whether we can // infer some from the child plans (static/recurisve terms). fn maintains_input_order(&self) -> Vec { @@ -131,22 +140,16 @@ impl ExecutionPlan for RecursiveQueryExec { ] } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn with_new_children( self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(RecursiveQueryExec { - name: self.name.clone(), - static_term: children[0].clone(), - recursive_term: children[1].clone(), - is_distinct: self.is_distinct, - work_table: self.work_table.clone(), - metrics: self.metrics.clone(), - })) + Ok(Arc::new(RecursiveQueryExec::try_new( + self.name.clone(), + children[0].clone(), + children[1].clone(), + self.is_distinct, + )?)) } fn execute( diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 07693f747fee..54b5a5404d24 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -37,14 +37,16 @@ use tokio::task::JoinHandle; use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; +use datafusion_physical_expr::PhysicalExpr; use crate::common::transpose; use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; use crate::repartition::distributor_channels::{channels, partition_aware_channels}; use crate::sorts::streaming_merge; -use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; +use crate::{ + DisplayFormatType, ExecutionPlan, Partitioning, PlanPropertiesCache, Statistics, +}; use super::common::{AbortOnDropMany, AbortOnDropSingle, SharedMemoryReservation}; use super::expressions::PhysicalSortExpr; @@ -308,6 +310,7 @@ pub struct RepartitionExec { /// Boolean flag to decide whether to preserve ordering. If true means /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. preserve_order: bool, + cache: PlanPropertiesCache, } #[derive(Debug, Clone)] @@ -413,9 +416,8 @@ impl ExecutionPlan for RepartitionExec { self } - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - self.input.schema() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -434,29 +436,10 @@ impl ExecutionPlan for RepartitionExec { Ok(Arc::new(repartition)) } - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) - } - fn benefits_from_input_partitioning(&self) -> Vec { vec![matches!(self.partitioning, Partitioning::Hash(_, _))] } - fn output_partitioning(&self) -> Partitioning { - self.partitioning.clone() - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - if self.maintains_input_order()[0] { - self.input().output_ordering() - } else { - None - } - } - fn maintains_input_order(&self) -> Vec { if self.preserve_order { vec![true] @@ -466,15 +449,6 @@ impl ExecutionPlan for RepartitionExec { } } - fn equivalence_properties(&self) -> EquivalenceProperties { - let mut result = self.input.equivalence_properties(); - // If the ordering is lost, reset the ordering equivalence class. - if !self.maintains_input_order()[0] { - result.clear_orderings(); - } - result - } - fn execute( &self, partition: usize, @@ -633,6 +607,7 @@ impl RepartitionExec { input: Arc, partitioning: Partitioning, ) -> Result { + let cache = PlanPropertiesCache::new_default(input.schema()); Ok(RepartitionExec { input, partitioning, @@ -642,7 +617,28 @@ impl RepartitionExec { })), metrics: ExecutionPlanMetricsSet::new(), preserve_order: false, - }) + cache, + } + .with_cache()) + } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let mut eq_properties = self.input.equivalence_properties().clone(); + // If the ordering is lost, reset the ordering equivalence class. + if !self.maintains_input_order()[0] { + eq_properties.clear_orderings(); + } + + // Output Partitioning + let output_partitioning = self.partitioning.clone(); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self } /// Specify if this reparititoning operation should preserve the order of @@ -659,7 +655,7 @@ impl RepartitionExec { // if there is only one input partition, merging is not required // to maintain order self.input.output_partitioning().partition_count() > 1; - self + self.with_cache() } /// Return the sort expressions that are used to merge diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 563a82f3ea7b..d59a3577acc0 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -66,14 +66,13 @@ use log::trace; use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::Result; use datafusion_execution::{RecordBatchStream, TaskContext}; -use datafusion_physical_expr::EquivalenceProperties; use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::sort::sort_batch; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - SendableRecordBatchStream, Statistics, + PlanPropertiesCache, SendableRecordBatchStream, Statistics, }; /// Partial Sort execution plan. @@ -93,6 +92,7 @@ pub struct PartialSortExec { preserve_partitioning: bool, /// Fetch highest/lowest n results fetch: Option, + cache: PlanPropertiesCache, } impl PartialSortExec { @@ -103,6 +103,7 @@ impl PartialSortExec { common_prefix_length: usize, ) -> Self { assert!(common_prefix_length > 0); + let cache = PlanPropertiesCache::new_default(input.schema()); Self { input, expr, @@ -110,7 +111,9 @@ impl PartialSortExec { metrics_set: ExecutionPlanMetricsSet::new(), preserve_partitioning: false, fetch: None, + cache, } + .with_cache() } /// Whether this `PartialSortExec` preserves partitioning of the children @@ -156,6 +159,30 @@ impl PartialSortExec { pub fn fetch(&self) -> Option { self.fetch } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + // Reset the ordering equivalence class with the new ordering: + let eq_properties = self + .input + .equivalence_properties() + .clone() + .with_reorder(self.expr.to_vec()); + + // Output Partitioning + let output_partitioning = if self.preserve_partitioning { + self.input.output_partitioning().clone() + } else { + Partitioning::UnknownPartitioning(1) + }; + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for PartialSortExec { @@ -184,28 +211,8 @@ impl ExecutionPlan for PartialSortExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() - } - - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - if self.preserve_partitioning { - self.input.output_partitioning() - } else { - Partitioning::UnknownPartitioning(1) - } - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - Some(&self.expr) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -220,13 +227,6 @@ impl ExecutionPlan for PartialSortExec { vec![false] } - fn equivalence_properties(&self) -> EquivalenceProperties { - // Reset the ordering equivalence class with the new ordering: - self.input - .equivalence_properties() - .with_reorder(self.expr.to_vec()) - } - fn children(&self) -> Vec> { vec![self.input.clone()] } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 2d8237011fff..b2f330784a1c 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -36,22 +36,22 @@ use crate::sorts::streaming_merge::streaming_merge; use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::topk::TopK; use crate::{ - DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, - Partitioning, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionMode, + ExecutionPlan, Partitioning, PlanPropertiesCache, SendableRecordBatchStream, + Statistics, }; use arrow::compute::{concat_batches, lexsort_to_indices, take}; use arrow::datatypes::SchemaRef; use arrow::ipc::reader::FileReader; use arrow::record_batch::RecordBatch; -use datafusion_common::{exec_err, plan_err, DataFusionError, Result}; +use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::{ human_readable_size, MemoryConsumer, MemoryReservation, }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::EquivalenceProperties; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; @@ -676,6 +676,7 @@ pub struct SortExec { preserve_partitioning: bool, /// Fetch highest/lowest n results fetch: Option, + cache: PlanPropertiesCache, } impl SortExec { @@ -692,13 +693,16 @@ impl SortExec { /// Create a new sort execution plan that produces a single, /// sorted output partition. pub fn new(expr: Vec, input: Arc) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); Self { expr, input, metrics_set: ExecutionPlanMetricsSet::new(), preserve_partitioning: false, fetch: None, + cache, } + .with_cache() } /// Create a new sort execution plan with the option to preserve @@ -732,7 +736,7 @@ impl SortExec { /// input partitions producing a single, sorted partition. pub fn with_preserve_partitioning(mut self, preserve_partitioning: bool) -> Self { self.preserve_partitioning = preserve_partitioning; - self + self.with_cache() } /// Modify how many rows to include in the result @@ -761,6 +765,35 @@ impl SortExec { pub fn fetch(&self) -> Option { self.fetch } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + // Reset the ordering equivalence class with the new ordering: + let eq_properties = self + .input + .equivalence_properties() + .clone() + .with_reorder(self.expr.to_vec()); + + // Output Partitioning + let output_partitioning = if self.preserve_partitioning { + self.input.output_partitioning().clone() + } else { + Partitioning::UnknownPartitioning(1) + }; + + // Execution Mode + let exec_mode = match self.input.unbounded_output() { + ExecutionMode::Unbounded | ExecutionMode::InExecutable => { + ExecutionMode::InExecutable + } + ExecutionMode::Bounded => ExecutionMode::Bounded, + }; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for SortExec { @@ -788,28 +821,8 @@ impl ExecutionPlan for SortExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() - } - - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - if self.preserve_partitioning { - self.input.output_partitioning() - } else { - Partitioning::UnknownPartitioning(1) - } - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - if children[0] { - plan_err!("Sort Error: Can not sort unbounded inputs.") - } else { - Ok(false) - } + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -830,17 +843,6 @@ impl ExecutionPlan for SortExec { vec![false] } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - Some(&self.expr) - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - // Reset the ordering equivalence class with the new ordering: - self.input - .equivalence_properties() - .with_reorder(self.expr.to_vec()) - } - fn with_new_children( self: Arc, children: Vec>, diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index f4b57e8bfb45..6a0ab8cb5748 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -26,14 +26,13 @@ use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::streaming_merge; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - SendableRecordBatchStream, Statistics, + PlanPropertiesCache, SendableRecordBatchStream, Statistics, }; -use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalSortRequirement; use log::{debug, trace}; @@ -74,17 +73,21 @@ pub struct SortPreservingMergeExec { metrics: ExecutionPlanMetricsSet, /// Optional number of rows to fetch. Stops producing rows after this fetch fetch: Option, + cache: PlanPropertiesCache, } impl SortPreservingMergeExec { /// Create a new sort execution plan pub fn new(expr: Vec, input: Arc) -> Self { + let cache = PlanPropertiesCache::new_default(input.schema()); Self { input, expr, metrics: ExecutionPlanMetricsSet::new(), fetch: None, + cache, } + .with_cache() } /// Sets the number of rows to fetch pub fn with_fetch(mut self, fetch: Option) -> Self { @@ -106,6 +109,21 @@ impl SortPreservingMergeExec { pub fn fetch(&self) -> Option { self.fetch } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = self.input.equivalence_properties().clone(); + + // Output Partitioning + let output_partitioning = Partitioning::UnknownPartitioning(1); + + // Execution Mode + let exec_mode = self.input.unbounded_output(); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for SortPreservingMergeExec { @@ -137,20 +155,8 @@ impl ExecutionPlan for SortPreservingMergeExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() - } - - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -165,18 +171,10 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![Some(PhysicalSortRequirement::from_sort_exprs(&self.expr))] } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input.output_ordering() - } - fn maintains_input_order(&self) -> Vec { vec![true] } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - fn children(&self) -> Vec> { vec![self.input.clone()] } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 59819c6921fb..920b687e40f8 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use super::{DisplayAs, DisplayFormatType}; +use super::{DisplayAs, DisplayFormatType, ExecutionMode, PlanPropertiesCache}; use crate::display::{OutputOrderingDisplay, ProjectSchemaDisplay}; use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; @@ -29,7 +29,7 @@ use arrow::datatypes::SchemaRef; use arrow_schema::Schema; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use async_trait::async_trait; use futures::stream::StreamExt; @@ -58,6 +58,7 @@ pub struct StreamingTableExec { projected_schema: SchemaRef, projected_output_ordering: Vec, infinite: bool, + cache: PlanPropertiesCache, } impl StreamingTableExec { @@ -84,14 +85,16 @@ impl StreamingTableExec { Some(p) => Arc::new(schema.project(p)?), None => schema, }; - + let cache = PlanPropertiesCache::new_default(projected_schema.clone()); Ok(Self { partitions, projected_schema, projection: projection.cloned().map(Into::into), projected_output_ordering: projected_output_ordering.into_iter().collect(), infinite, - }) + cache, + } + .with_cache()) } pub fn partitions(&self) -> &Vec> { @@ -117,6 +120,29 @@ impl StreamingTableExec { pub fn is_infinite(&self) -> bool { self.infinite } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ); + + // Output Partitioning + let output_partitioning = + Partitioning::UnknownPartitioning(self.partitions.len()); + + // Execution Mode + let exec_mode = if self.infinite { + ExecutionMode::Unbounded + } else { + ExecutionMode::Bounded + }; + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl std::fmt::Debug for StreamingTableExec { @@ -149,18 +175,28 @@ impl DisplayAs for StreamingTableExec { write!(f, ", infinite_source=true")?; } - self.projected_output_ordering - .first() - .map_or(Ok(()), |ordering| { - if !ordering.is_empty() { - write!( - f, - ", output_ordering={}", - OutputOrderingDisplay(ordering) - )?; + let orderings = &self.projected_output_ordering; + if let Some(ordering) = orderings.first() { + if !ordering.is_empty() { + let start = if orderings.len() == 1 { + ", output_ordering=" + } else { + ", output_orderings=[" + }; + write!(f, "{}", start)?; + for (idx, ordering) in + orderings.iter().enumerate().filter(|(_, o)| !o.is_empty()) + { + match idx { + 0 => write!(f, "{}", OutputOrderingDisplay(ordering))?, + _ => write!(f, ", {}", OutputOrderingDisplay(ordering))?, + } } - Ok(()) - }) + let end = if orderings.len() == 1 { "" } else { "]" }; + write!(f, "{}", end)?; + } + } + Ok(()) } } } @@ -172,29 +208,8 @@ impl ExecutionPlan for StreamingTableExec { self } - fn schema(&self) -> SchemaRef { - self.projected_schema.clone() - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.partitions.len()) - } - - fn unbounded_output(&self, _children: &[bool]) -> Result { - Ok(self.infinite) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.projected_output_ordering - .first() - .map(|ordering| ordering.as_slice()) - } - - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 5a8ef2db77c2..cfbbd054bac2 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -26,15 +26,14 @@ use std::{ use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::{ - common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + common, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortExpr; use futures::Stream; use tokio::sync::Barrier; @@ -121,6 +120,7 @@ pub struct MockExec { /// if true (the default), sends data using a separate task to to ensure the /// batches are not available without this stream yielding first use_task: bool, + cache: PlanPropertiesCache, } impl MockExec { @@ -132,11 +132,14 @@ impl MockExec { /// ensure any poll loops are correct. This behavior can be /// changed with `with_use_task` pub fn new(data: Vec>, schema: SchemaRef) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); Self { data, schema, use_task: true, + cache, } + .with_cache() } /// If `use_task` is true (the default) then the batches are sent @@ -146,6 +149,19 @@ impl MockExec { self.use_task = use_task; self } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for MockExec { @@ -167,20 +183,12 @@ impl ExecutionPlan for MockExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { - unimplemented!() + vec![] } fn with_new_children( @@ -276,6 +284,7 @@ pub struct BarrierExec { /// all streams wait on this barrier to produce barrier: Arc, + cache: PlanPropertiesCache, } impl BarrierExec { @@ -283,11 +292,14 @@ impl BarrierExec { pub fn new(data: Vec>, schema: SchemaRef) -> Self { // wait for all streams and the input let barrier = Arc::new(Barrier::new(data.len() + 1)); + let cache = PlanPropertiesCache::new_default(schema.clone()); Self { data, schema, barrier, + cache, } + .with_cache() } /// wait until all the input streams and this function is ready @@ -296,6 +308,20 @@ impl BarrierExec { self.barrier.wait().await; println!("BarrierExec::wait done waiting"); } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache + .with_partitioning(Partitioning::UnknownPartitioning(self.data.len())); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for BarrierExec { @@ -317,16 +343,8 @@ impl ExecutionPlan for BarrierExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.data.len()) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -383,7 +401,7 @@ impl ExecutionPlan for BarrierExec { /// A mock execution plan that errors on a call to execute #[derive(Debug)] pub struct ErrorExec { - schema: SchemaRef, + cache: PlanPropertiesCache, } impl Default for ErrorExec { @@ -399,7 +417,21 @@ impl ErrorExec { DataType::Int64, true, )])); - Self { schema } + let cache = PlanPropertiesCache::new_default(schema.clone()); + Self { cache }.with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self } } @@ -422,16 +454,8 @@ impl ExecutionPlan for ErrorExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -460,6 +484,7 @@ impl ExecutionPlan for ErrorExec { pub struct StatisticsExec { stats: Statistics, schema: Arc, + cache: PlanPropertiesCache, } impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { @@ -468,10 +493,26 @@ impl StatisticsExec { .column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); + let cache = PlanPropertiesCache::new_default(Arc::new(schema.clone())); Self { stats, schema: Arc::new(schema), + cache, } + .with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(2)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self } } @@ -499,16 +540,8 @@ impl ExecutionPlan for StatisticsExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(2) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -548,16 +581,20 @@ pub struct BlockingExec { /// Ref-counting helper to check if the plan and the produced stream are still in memory. refs: Arc<()>, + cache: PlanPropertiesCache, } impl BlockingExec { /// Create new [`BlockingExec`] with a give schema and number of partitions. pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); Self { schema, n_partitions, refs: Default::default(), + cache, } + .with_cache() } /// Weak pointer that can be used for ref-counting this execution plan and its streams. @@ -568,6 +605,20 @@ impl BlockingExec { pub fn refs(&self) -> Weak<()> { Arc::downgrade(&self.refs) } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache + .with_partitioning(Partitioning::UnknownPartitioning(self.n_partitions)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for BlockingExec { @@ -589,8 +640,8 @@ impl ExecutionPlan for BlockingExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -598,14 +649,6 @@ impl ExecutionPlan for BlockingExec { vec![] } - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.n_partitions) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn with_new_children( self: Arc, _: Vec>, @@ -681,16 +724,20 @@ pub struct PanicExec { /// Number of output partitions. Each partition will produce this /// many empty output record batches prior to panicing batches_until_panics: Vec, + cache: PlanPropertiesCache, } impl PanicExec { /// Create new [`PanicExec`] with a give schema and number of /// partitions, which will each panic immediately. pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); Self { schema, batches_until_panics: vec![0; n_partitions], + cache, } + .with_cache() } /// Set the number of batches prior to panic for a partition @@ -698,6 +745,21 @@ impl PanicExec { self.batches_until_panics[partition] = count; self } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + let num_partitions = self.batches_until_panics.len(); + new_cache = new_cache + .with_partitioning(Partitioning::UnknownPartitioning(num_partitions)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for PanicExec { @@ -719,8 +781,8 @@ impl ExecutionPlan for PanicExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { @@ -728,15 +790,6 @@ impl ExecutionPlan for PanicExec { vec![] } - fn output_partitioning(&self) -> Partitioning { - let num_partitions = self.batches_until_panics.len(); - Partitioning::UnknownPartitioning(num_partitions) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn with_new_children( self: Arc, _: Vec>, diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index d01ea5507449..974ea630b160 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -27,12 +27,11 @@ use std::task::{Context, Poll}; use std::{any::Any, sync::Arc}; use super::{ - expressions::PhysicalSortExpr, + exec_mode_flatten, metrics::{ExecutionPlanMetricsSet, MetricsSet}, ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, Statistics, + PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::common::get_meet_of_orderings; use crate::metrics::BaselineMetrics; use crate::stream::ObservedStream; @@ -91,26 +90,82 @@ pub struct UnionExec { inputs: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Schema of Union - schema: SchemaRef, + cache: PlanPropertiesCache, } impl UnionExec { /// Create a new UnionExec pub fn new(inputs: Vec>) -> Self { let schema = union_schema(&inputs); - + let cache = PlanPropertiesCache::new_default(schema); UnionExec { inputs, metrics: ExecutionPlanMetricsSet::new(), - schema, + cache, } + .with_cache() } /// Get inputs of the execution plan pub fn inputs(&self) -> &Vec> { &self.inputs } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + // TODO: In some cases, we should be able to preserve some equivalence + // classes and constants. Add support for such cases. + let children_eqs = self + .inputs + .iter() + .map(|child| child.equivalence_properties()) + .collect::>(); + let mut eq_properties = EquivalenceProperties::new(self.schema()); + // Use the ordering equivalence class of the first child as the seed: + let mut meets = children_eqs[0] + .oeq_class() + .iter() + .map(|item| item.to_vec()) + .collect::>(); + // Iterate over all the children: + for child_eqs in &children_eqs[1..] { + // Compute meet orderings of the current meets and the new ordering + // equivalence class. + let mut idx = 0; + while idx < meets.len() { + // Find all the meets of `current_meet` with this child's orderings: + let valid_meets = child_eqs.oeq_class().iter().filter_map(|ordering| { + child_eqs.get_meet_ordering(ordering, &meets[idx]) + }); + // Use the longest of these meets as others are redundant: + if let Some(next_meet) = valid_meets.max_by_key(|m| m.len()) { + meets[idx] = next_meet; + idx += 1; + } else { + meets.swap_remove(idx); + } + } + } + // We know have all the valid orderings after union, remove redundant + // entries (implicitly) and return: + eq_properties.add_new_orderings(meets); + + // Output Partitioning + // Output the combination of all output partitions of the inputs if the Union is not partition aware + let num_partitions = self + .inputs + .iter() + .map(|plan| plan.output_partitioning().partition_count()) + .sum(); + let output_partitioning = Partitioning::UnknownPartitioning(num_partitions); + + // Execution Mode + let exec_mode = exec_mode_flatten(self.inputs.iter()); + + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self + } } impl DisplayAs for UnionExec { @@ -133,40 +188,14 @@ impl ExecutionPlan for UnionExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children.iter().any(|x| *x)) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { self.inputs.clone() } - /// Output of the union is the combination of all output partitions of the inputs - fn output_partitioning(&self) -> Partitioning { - // Output the combination of all output partitions of the inputs if the Union is not partition aware - let num_partitions = self - .inputs - .iter() - .map(|plan| plan.output_partitioning().partition_count()) - .sum(); - - Partitioning::UnknownPartitioning(num_partitions) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - // The output ordering is the "meet" of its input orderings. - // The meet is the finest ordering that satisfied by all the input - // orderings, see https://en.wikipedia.org/wiki/Join_and_meet. - get_meet_of_orderings(&self.inputs) - } - fn maintains_input_order(&self) -> Vec { // If the Union has an output ordering, it maintains at least one // child's ordering (i.e. the meet). @@ -192,46 +221,6 @@ impl ExecutionPlan for UnionExec { } } - fn equivalence_properties(&self) -> EquivalenceProperties { - // TODO: In some cases, we should be able to preserve some equivalence - // classes and constants. Add support for such cases. - let children_eqs = self - .inputs - .iter() - .map(|child| child.equivalence_properties()) - .collect::>(); - let mut result = EquivalenceProperties::new(self.schema()); - // Use the ordering equivalence class of the first child as the seed: - let mut meets = children_eqs[0] - .oeq_class() - .iter() - .map(|item| item.to_vec()) - .collect::>(); - // Iterate over all the children: - for child_eqs in &children_eqs[1..] { - // Compute meet orderings of the current meets and the new ordering - // equivalence class. - let mut idx = 0; - while idx < meets.len() { - // Find all the meets of `current_meet` with this child's orderings: - let valid_meets = child_eqs.oeq_class().iter().filter_map(|ordering| { - child_eqs.get_meet_ordering(ordering, &meets[idx]) - }); - // Use the longest of these meets as others are redundant: - if let Some(next_meet) = valid_meets.max_by_key(|m| m.len()) { - meets[idx] = next_meet; - idx += 1; - } else { - meets.swap_remove(idx); - } - } - } - // We know have all the valid orderings after union, remove redundant - // entries (implicitly) and return: - result.add_new_orderings(meets); - result - } - fn with_new_children( self: Arc, children: Vec>, @@ -328,8 +317,7 @@ pub struct InterleaveExec { inputs: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Schema of Interleave - schema: SchemaRef, + cache: PlanPropertiesCache, } impl InterleaveExec { @@ -342,18 +330,34 @@ impl InterleaveExec { "Not all InterleaveExec children have a consistent hash partitioning" ); } - + let cache = PlanPropertiesCache::new_default(schema); Ok(InterleaveExec { inputs, metrics: ExecutionPlanMetricsSet::new(), - schema, - }) + cache, + } + .with_cache()) } /// Get inputs of the execution plan pub fn inputs(&self) -> &Vec> { &self.inputs } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + + // Output Partitioning + let output_partitioning = self.inputs[0].output_partitioning().clone(); + new_cache = new_cache.with_partitioning(output_partitioning); + + // Execution Mode + let exec_mode = exec_mode_flatten(self.inputs.iter()); + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for InterleaveExec { @@ -376,32 +380,14 @@ impl ExecutionPlan for InterleaveExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children.iter().any(|x| *x)) + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { self.inputs.clone() } - /// All inputs must have the same partitioning. The output partioning of InterleaveExec is the same as the inputs - /// (NOT combined). E.g. if there are 10 inputs where each is `Hash(3)`-partitioned, InterleaveExec is also - /// `Hash(3)`-partitioned. - fn output_partitioning(&self) -> Partitioning { - self.inputs[0].output_partitioning() - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn maintains_input_order(&self) -> Vec { vec![false; self.inputs().len()] } @@ -485,8 +471,8 @@ pub fn can_interleave>>( let reference = first.borrow().output_partitioning(); matches!(reference, Partitioning::Hash(_, _)) && inputs - .map(|plan| plan.borrow().output_partitioning()) - .all(|partition| partition == reference) + .map(|plan| plan.borrow().output_partitioning().clone()) + .all(|partition| partition == *reference) } fn union_schema(inputs: &[Arc]) -> SchemaRef { @@ -614,7 +600,7 @@ mod tests { use arrow_schema::{DataType, SortOptions}; use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::PhysicalExpr; + use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) fn create_test_schema() -> Result { diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index b9e732c317af..64c575831562 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -19,10 +19,10 @@ //! type, conceptually is like joining each row with all the values in the list column. use std::{any::Any, sync::Arc}; -use super::DisplayAs; +use super::{DisplayAs, PlanPropertiesCache}; use crate::{ - expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PhysicalExpr, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, + expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, PhysicalExpr, + RecordBatchStream, SendableRecordBatchStream, }; use arrow::array::{ @@ -60,6 +60,7 @@ pub struct UnnestExec { options: UnnestOptions, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl UnnestExec { @@ -70,13 +71,28 @@ impl UnnestExec { schema: SchemaRef, options: UnnestOptions, ) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); UnnestExec { input, schema, column, options, metrics: Default::default(), + cache, } + .with_cache() + } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(self.input.output_partitioning().clone()); + + // Execution Mode + new_cache = new_cache.with_exec_mode(self.input.unbounded_output()); + + self.cache = new_cache; + self } } @@ -99,21 +115,14 @@ impl ExecutionPlan for UnnestExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.input.clone()] } - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) - } - fn with_new_children( self: Arc, children: Vec>, @@ -130,14 +139,6 @@ impl ExecutionPlan for UnnestExec { vec![Distribution::UnspecifiedDistribution] } - fn output_partitioning(&self) -> Partitioning { - self.input.output_partitioning() - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn execute( &self, partition: usize, diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index f82f7ea2f869..6351ad8f5b10 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -20,8 +20,10 @@ use std::any::Any; use std::sync::Arc; -use super::expressions::PhysicalSortExpr; -use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; +use super::{ + common, DisplayAs, ExecutionMode, PlanPropertiesCache, SendableRecordBatchStream, + Statistics, +}; use crate::{ memory::MemoryStream, ColumnarValue, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, @@ -39,6 +41,7 @@ pub struct ValuesExec { schema: SchemaRef, /// The data data: Vec, + cache: PlanPropertiesCache, } impl ValuesExec { @@ -85,7 +88,7 @@ impl ValuesExec { .collect::>>()?; let batch = RecordBatch::try_new(schema.clone(), arr)?; let data: Vec = vec![batch]; - Ok(Self { schema, data }) + Self::try_new_from_batches(schema, data) } /// Create a new plan using the provided schema and batches. @@ -109,16 +112,32 @@ impl ValuesExec { } } + let cache = PlanPropertiesCache::new_default(schema.clone()); Ok(ValuesExec { schema, data: batches, - }) + cache, + } + .with_cache()) } /// provides the data pub fn data(&self) -> Vec { self.data.clone() } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + let exec_mode = ExecutionMode::Bounded; + new_cache = new_cache.with_exec_mode(exec_mode); + + self.cache = new_cache; + self + } } impl DisplayAs for ValuesExec { @@ -141,30 +160,22 @@ impl ExecutionPlan for ValuesExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } fn with_new_children( self: Arc, _: Vec>, ) -> Result> { - Ok(Arc::new(ValuesExec { - schema: self.schema.clone(), - data: self.data.clone(), - })) + Ok(Arc::new(ValuesExec::try_new_from_batches( + self.schema.clone(), + self.data.clone(), + )?)) } fn execute( 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 9d247d689c07..d3b9ff74b644 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -35,7 +35,7 @@ use crate::windows::{ }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, - InputOrderMode, Partitioning, RecordBatchStream, SendableRecordBatchStream, + InputOrderMode, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; @@ -58,9 +58,7 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; -use datafusion_physical_expr::{ - EquivalenceProperties, PhysicalExpr, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use ahash::RandomState; use futures::stream::Stream; @@ -91,6 +89,7 @@ pub struct BoundedWindowAggExec { // `ordered_partition_by_indices` would be 0, 1. // See `get_ordered_partition_by_indices` for more details. ordered_partition_by_indices: Vec, + cache: PlanPropertiesCache, } impl BoundedWindowAggExec { @@ -121,7 +120,8 @@ impl BoundedWindowAggExec { vec![] } }; - Ok(Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); + let window = Self { input, window_expr, schema, @@ -129,7 +129,9 @@ impl BoundedWindowAggExec { metrics: ExecutionPlanMetricsSet::new(), input_order_mode, ordered_partition_by_indices, - }) + cache, + }; + Ok(window.with_cache()) } /// Window expressions @@ -179,6 +181,28 @@ impl BoundedWindowAggExec { } }) } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = + window_equivalence_properties(&self.schema, &self.input, &self.window_expr); + + // As we can have repartitioning using the partition keys, this can + // be either one or more than one, depending on the presence of + // repartitioning. + let output_partitioning = self.input.output_partitioning().clone(); + + // unbounded output + let unbounded_output = self.input.unbounded_output(); + + // Construct properties cache + self.cache = PlanPropertiesCache::new( + eq_properties, + output_partitioning, + unbounded_output, + ); + self + } } impl DisplayAs for BoundedWindowAggExec { @@ -216,30 +240,14 @@ impl ExecutionPlan for BoundedWindowAggExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.input.clone()] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - // As we can have repartitioning using the partition keys, this can - // be either one or more than one, depending on the presence of - // repartitioning. - self.input.output_partitioning() - } - - fn unbounded_output(&self, children: &[bool]) -> Result { - Ok(children[0]) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input().output_ordering() - } - fn required_input_ordering(&self) -> Vec>> { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); @@ -265,11 +273,6 @@ impl ExecutionPlan for BoundedWindowAggExec { } } - /// Get the [`EquivalenceProperties`] within the plan - fn equivalence_properties(&self) -> EquivalenceProperties { - window_equivalence_properties(&self.schema, &self.input, &self.window_expr) - } - fn maintains_input_order(&self) -> Vec { vec![true] } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 01818405b810..4143b5c9f508 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -27,7 +27,7 @@ use crate::{ cume_dist, dense_rank, lag, lead, percent_rank, rank, Literal, NthValue, Ntile, PhysicalSortExpr, RowNumber, }, - udaf, unbounded_output, ExecutionPlan, InputOrderMode, PhysicalExpr, + udaf, ExecutionPlan, InputOrderMode, PhysicalExpr, }; use arrow::datatypes::Schema; @@ -372,8 +372,8 @@ pub(crate) fn window_equivalence_properties( ) -> EquivalenceProperties { // We need to update the schema, so we can not directly use // `input.equivalence_properties()`. - let mut window_eq_properties = - EquivalenceProperties::new(schema.clone()).extend(input.equivalence_properties()); + let mut window_eq_properties = EquivalenceProperties::new(schema.clone()) + .extend(input.equivalence_properties().clone()); for expr in window_expr { if let Some(builtin_window_expr) = @@ -415,7 +415,7 @@ pub fn get_best_fitting_window( } else { return Ok(None); }; - let is_unbounded = unbounded_output(input); + let is_unbounded = input.unbounded_output().is_unbounded(); if !is_unbounded && input_order_mode != InputOrderMode::Sorted { // Executor has bounded input and `input_order_mode` is not `InputOrderMode::Sorted` // in this case removing the sort is not helpful, return: @@ -477,7 +477,7 @@ pub fn get_window_mode( orderby_keys: &[PhysicalSortExpr], input: &Arc, ) -> Option<(bool, InputOrderMode)> { - let input_eqs = input.equivalence_properties(); + let input_eqs = input.equivalence_properties().clone(); let mut partition_by_reqs: Vec = vec![]; let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); partition_by_reqs.extend(indices.iter().map(|&idx| PhysicalSortRequirement { diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 6c245f65ba4f..9b454e38f4e3 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -30,9 +30,9 @@ use crate::windows::{ window_equivalence_properties, }; use crate::{ - ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, - Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, - WindowExpr, + ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, + ExecutionPlan, PhysicalExpr, PlanPropertiesCache, RecordBatchStream, + SendableRecordBatchStream, Statistics, WindowExpr, }; use arrow::compute::{concat, concat_batches}; @@ -45,9 +45,9 @@ use arrow::{ }; use datafusion_common::stats::Precision; use datafusion_common::utils::evaluate_partition_ranges; -use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; +use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalSortRequirement; use futures::stream::Stream; use futures::{ready, StreamExt}; @@ -68,6 +68,7 @@ pub struct WindowAggExec { /// Partition by indices that defines preset for existing ordering // see `get_ordered_partition_by_indices` for more details. ordered_partition_by_indices: Vec, + cache: PlanPropertiesCache, } impl WindowAggExec { @@ -82,14 +83,17 @@ impl WindowAggExec { let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); - Ok(Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); + let window = Self { input, window_expr, schema, partition_keys, metrics: ExecutionPlanMetricsSet::new(), ordered_partition_by_indices, - }) + cache, + }; + Ok(window.with_cache()) } /// Window expressions @@ -115,6 +119,34 @@ impl WindowAggExec { &self.ordered_partition_by_indices, ) } + + fn with_cache(mut self) -> Self { + // Equivalence Properties + let eq_properties = + window_equivalence_properties(&self.schema, &self.input, &self.window_expr); + + // output partitioning + // because we can have repartitioning using the partition keys + // this would be either 1 or more than 1 depending on the presense of + // repartitioning + let output_partitioning = self.input.output_partitioning().clone(); + + // unbounded output + let unbounded_output = match self.input.unbounded_output() { + ExecutionMode::Bounded => ExecutionMode::Bounded, + ExecutionMode::Unbounded | ExecutionMode::InExecutable => { + ExecutionMode::InExecutable + } + }; + + // Construct properties cache + self.cache = PlanPropertiesCache::new( + eq_properties, + output_partitioning, + unbounded_output, + ); + self + } } impl DisplayAs for WindowAggExec { @@ -151,39 +183,14 @@ impl ExecutionPlan for WindowAggExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![self.input.clone()] } - /// Get the output partitioning of this plan - fn output_partitioning(&self) -> Partitioning { - // because we can have repartitioning using the partition keys - // this would be either 1 or more than 1 depending on the presense of - // repartitioning - self.input.output_partitioning() - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn unbounded_output(&self, children: &[bool]) -> Result { - if children[0] { - plan_err!( - "Window Error: Windowing is not currently support for unbounded inputs." - ) - } else { - Ok(false) - } - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.input().output_ordering() - } - fn maintains_input_order(&self) -> Vec { vec![true] } @@ -210,11 +217,6 @@ impl ExecutionPlan for WindowAggExec { } } - /// Get the [`EquivalenceProperties`] within the plan - fn equivalence_properties(&self) -> EquivalenceProperties { - window_equivalence_properties(&self.schema, &self.input, &self.window_expr) - } - fn with_new_children( self: Arc, children: Vec>, diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index c74a596f3dae..13bc98f1878b 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -26,9 +26,9 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::Partitioning; use crate::memory::MemoryStream; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan}; - -use super::expressions::PhysicalSortExpr; +use crate::{ + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, PlanPropertiesCache, +}; use super::{ metrics::{ExecutionPlanMetricsSet, MetricsSet}, @@ -85,17 +85,21 @@ pub struct WorkTableExec { work_table: Arc, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanPropertiesCache, } impl WorkTableExec { /// Create a new execution plan for a worktable exec. pub fn new(name: String, schema: SchemaRef) -> Self { + let cache = PlanPropertiesCache::new_default(schema.clone()); Self { name, schema, metrics: ExecutionPlanMetricsSet::new(), work_table: Arc::new(WorkTable::new()), + cache, } + .with_cache() } pub(super) fn with_work_table(&self, work_table: Arc) -> Self { @@ -104,8 +108,21 @@ impl WorkTableExec { schema: self.schema.clone(), metrics: ExecutionPlanMetricsSet::new(), work_table, + cache: self.cache.clone(), } } + + fn with_cache(mut self) -> Self { + let mut new_cache = self.cache; + // Output Partitioning + new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + + // Execution Mode + new_cache = new_cache.with_exec_mode(ExecutionMode::Bounded); + + self.cache = new_cache; + self + } } impl DisplayAs for WorkTableExec { @@ -127,18 +144,14 @@ impl ExecutionPlan for WorkTableExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn cache(&self) -> &PlanPropertiesCache { + &self.cache } fn children(&self) -> Vec> { vec![] } - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - fn maintains_input_order(&self) -> Vec { vec![false] } @@ -147,10 +160,6 @@ impl ExecutionPlan for WorkTableExec { vec![false] } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - fn with_new_children( self: Arc, _: Vec>, diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index fa4445d4cd4c..3cb7cee0b913 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3580,9 +3580,9 @@ SortPreservingMergeExec: [c@3 ASC NULLS LAST] --ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] ----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow }], mode=[Linear] ------CoalesceBatchesExec: target_batch_size=4096 ---------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST +--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST] +------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] # CTAS with NTILE function statement ok From 27b6805c25c07f980690393122d1cd2c7e3c7d7d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 15 Feb 2024 17:22:11 +0300 Subject: [PATCH 02/15] Update comments --- .../physical_optimizer/pipeline_checker.rs | 2 +- .../replace_with_order_preserving_variants.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 2 +- .../physical-plan/src/joins/cross_join.rs | 2 +- .../physical-plan/src/joins/hash_join.rs | 2 +- datafusion/physical-plan/src/lib.rs | 72 +++++++++++++++---- datafusion/physical-plan/src/sorts/sort.rs | 4 +- .../src/windows/window_agg_exec.rs | 4 +- 8 files changed, 69 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 780a09f24380..9f694a5c7695 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -78,7 +78,7 @@ pub fn check_finiteness_requirements( the 'allow_symmetric_joins_without_pruning' configuration flag"); } } - if !input.unbounded_output().is_executable() { + if !input.unbounded_output().pipeline_friendly() { Err(plan_datafusion_err!( "Cannot execute pipeline breaking queries, operator: {:?}", input 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 a03cb40e8a13..5fc24205f5da 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 @@ -241,7 +241,7 @@ pub(crate) fn replace_with_order_preserving_variants( // For unbounded cases, we replace with the order-preserving variant in any // case, as doing so helps fix the pipeline. Also replace if config allows. let use_order_preserving_variant = config.optimizer.prefer_existing_sort - || !requirements.plan.unbounded_output().is_executable(); + || !requirements.plan.unbounded_output().pipeline_friendly(); // Create an alternate plan with order-preserving variants: let mut alternate_plan = plan_with_order_preserving_variants( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 9d880e461efb..a686f8f462d6 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -542,7 +542,7 @@ impl AggregateExec { && self.input_order_mode == InputOrderMode::Linear { // Cannot run without breaking pipeline. - unbounded_output = ExecutionMode::InExecutable; + unbounded_output = ExecutionMode::PipelineBreaking; } self.cache = PlanPropertiesCache::new( diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index f23c78c18515..0f4fea1cab96 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -125,7 +125,7 @@ impl CrossJoinExec { let exec_mode = match (left_unbounded, right_unbounded) { (ExecutionMode::Bounded, ExecutionMode::Bounded) => ExecutionMode::Bounded, // If any of the inputs is unbounded, cross join break pipeline. - (_, _) => ExecutionMode::InExecutable, + (_, _) => ExecutionMode::PipelineBreaking, }; self.cache = diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index aee207dc2871..8fb36b728ecb 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -461,7 +461,7 @@ impl HashJoinExec { )); let exec_mode = if breaking { - ExecutionMode::InExecutable + ExecutionMode::PipelineBreaking } else { exec_mode_flatten([left, right]) }; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 6805b8448df8..e7e451586fbc 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -450,19 +450,45 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } } +/// Describes the execution mode of each operator's resulting stream. +/// +/// This enum defines the possible execution modes: +/// +/// - `Bounded`: Represents a mode where execution is limited or constrained within certain bounds. +/// In this mode, the process operates within defined limits or boundaries. +/// +/// - `Unbounded`: Indicates a mode where execution is not limited by any specific bounds. +/// Processes or tasks in this mode can operate without constraints or limitations. +/// +/// - `PipelineBreaking`: Denotes a mode where execution can potentially break pipeline constraints. +/// This mode may disrupt established pipelines or processes that rely on sequential operation. +/// +/// This enum can be used to specify the behavior or characteristics of a process or task +/// in various execution scenarios. #[derive(Clone, Copy, PartialEq, Debug)] pub enum ExecutionMode { + /// Represents the mode where generated stream is bounded, e.g. finite. Bounded, + /// Represents the mode where generated stream is unbounded, e.g. infinite. + /// Operator can generate streaming results with bounded memory. + /// In this mode, execution can still continue successfully. Unbounded, - InExecutable, + /// Represents the mode, where input stream to the operator is unbounded. However, + /// operator cannot generate streaming results from streaming inputs. In this case, + /// execution mode will be pipeline breaking. e.g. operator requires unbounded memory + /// to generate its result. + PipelineBreaking, } impl ExecutionMode { + /// Check whether the execution mode is unbounded or not. pub fn is_unbounded(&self) -> bool { matches!(self, ExecutionMode::Unbounded) } - pub fn is_executable(&self) -> bool { + /// Check whether the execution is pipeline friendly. If so, operator can execute + /// safely. + pub fn pipeline_friendly(&self) -> bool { matches!(self, ExecutionMode::Bounded | ExecutionMode::Unbounded) } } @@ -471,10 +497,11 @@ fn exec_mode_flatten_helper(modes: &[ExecutionMode]) -> ExecutionMode { let mut result = ExecutionMode::Bounded; for mode in modes { match (mode, result) { - (ExecutionMode::InExecutable, _) | (_, ExecutionMode::InExecutable) => { - // If any of the modes is `InExecutable`. result is `InExecutable` + (ExecutionMode::PipelineBreaking, _) + | (_, ExecutionMode::PipelineBreaking) => { + // If any of the modes is `InExecutable`. result is `PipelineBreaking` // Return immediately - return ExecutionMode::InExecutable; + return ExecutionMode::PipelineBreaking; } ( ExecutionMode::Unbounded, @@ -493,6 +520,9 @@ fn exec_mode_flatten_helper(modes: &[ExecutionMode]) -> ExecutionMode { result } +/// Constructs execution mode of the operator using its children. +/// This util assumes operator applied is pipeline friendly. For operators +/// not pipeline friendly, `exec_mode_safe_flatten` should be used. fn exec_mode_flatten<'a>( children: impl IntoIterator>, ) -> ExecutionMode { @@ -507,21 +537,23 @@ fn exec_mode_safe_flatten_helper(modes: &[ExecutionMode]) -> ExecutionMode { let mut result = ExecutionMode::Bounded; for mode in modes { match (mode, result) { - (ExecutionMode::Unbounded | ExecutionMode::InExecutable, _) - | (_, ExecutionMode::Unbounded | ExecutionMode::InExecutable) => { - // If any of the modes is `InExecutable`. result is `InExecutable` - // Return immediately - return ExecutionMode::InExecutable; - } (ExecutionMode::Bounded, ExecutionMode::Bounded) => { // When both modes are bounded, result is bounded result = ExecutionMode::Bounded; } + (_, _) => { + // If any of the modes is `InExecutable` or `Unbounded`. result is `PipelineBreaking` + // Return immediately + return ExecutionMode::PipelineBreaking; + } } } result } +/// Constructs execution mode of the operator using its children. +/// This util assumes operator applied is not pipeline friendly. For operators +/// that pipeline friendly, `exec_mode_flatten` should be used. fn exec_mode_safe_flatten<'a>( children: impl IntoIterator>, ) -> ExecutionMode { @@ -532,20 +564,30 @@ fn exec_mode_safe_flatten<'a>( exec_mode_safe_flatten_helper(&modes) } +/// Represents a cache for plan properties used in query optimization. +/// +/// This struct holds various properties useful for the query planning, which are used +/// during optimization and execution phases. #[derive(Debug, Clone)] pub struct PlanPropertiesCache { + /// Stores Equivalence Properties of the [`ExecutionPlan`]. See [`EquivalenceProperties`] pub eq_properties: EquivalenceProperties, + /// Stores Output Partitioning of the [`ExecutionPlan`]. See [`Partitioning`] pub partitioning: Partitioning, + /// Stores Execution Mode of the [`ExecutionPlan`]. See [`ExecutionMode`] pub exec_mode: ExecutionMode, + /// Stores output ordering of the [`ExecutionPlan`]. `None` represents, no ordering. output_ordering: Option, } impl PlanPropertiesCache { + /// Construct a new `PlanPropertiesCache` from the pub fn new( eq_properties: EquivalenceProperties, partitioning: Partitioning, exec_mode: ExecutionMode, ) -> Self { + // Output ordering can be derived from `eq_properties`. let output_ordering = eq_properties.oeq_class().output_ordering(); Self { eq_properties, @@ -555,11 +597,13 @@ impl PlanPropertiesCache { } } + /// Construct a default `PlanPropertiesCache`, for a given schema. pub fn new_default(schema: SchemaRef) -> PlanPropertiesCache { // Defaults are most restrictive possible values. let eq_properties = EquivalenceProperties::new(schema); + // Please note that this default is not safe, and should be overwritten. let partitioning = Partitioning::UnknownPartitioning(0); - let exec_mode = ExecutionMode::InExecutable; + let exec_mode = ExecutionMode::PipelineBreaking; let output_ordering = None; Self { eq_properties, @@ -569,17 +613,21 @@ impl PlanPropertiesCache { } } + /// Overwrite partitioning with its new value pub fn with_partitioning(mut self, partitioning: Partitioning) -> Self { self.partitioning = partitioning; self } + /// Overwrite Execution Mode with its new value pub fn with_exec_mode(mut self, exec_mode: ExecutionMode) -> Self { self.exec_mode = exec_mode; self } + /// Overwrite Equivalence Properties with its new value pub fn with_eq_properties(mut self, eq_properties: EquivalenceProperties) -> Self { + // Changing equivalence properties, changes output ordering also. Make sure to overwrite it. self.output_ordering = eq_properties.oeq_class().output_ordering(); self.eq_properties = eq_properties; self diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index b2f330784a1c..e97c550cdcd4 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -784,8 +784,8 @@ impl SortExec { // Execution Mode let exec_mode = match self.input.unbounded_output() { - ExecutionMode::Unbounded | ExecutionMode::InExecutable => { - ExecutionMode::InExecutable + ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { + ExecutionMode::PipelineBreaking } ExecutionMode::Bounded => ExecutionMode::Bounded, }; diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 9b454e38f4e3..ddfce03e2026 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -134,8 +134,8 @@ impl WindowAggExec { // unbounded output let unbounded_output = match self.input.unbounded_output() { ExecutionMode::Bounded => ExecutionMode::Bounded, - ExecutionMode::Unbounded | ExecutionMode::InExecutable => { - ExecutionMode::InExecutable + ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { + ExecutionMode::PipelineBreaking } }; From c8cece8f9ae097ef99484b2c789c2e6928dba87b Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 21 Feb 2024 17:02:00 +0300 Subject: [PATCH 03/15] Review Part 1 --- datafusion-cli/src/exec.rs | 15 +- .../examples/custom_datasource.rs | 16 +- datafusion/core/src/dataframe/mod.rs | 8 +- .../datasource/physical_plan/arrow_file.rs | 16 +- .../core/src/datasource/physical_plan/avro.rs | 21 +- .../core/src/datasource/physical_plan/csv.rs | 20 +- .../datasource/physical_plan/file_stream.rs | 5 +- .../core/src/datasource/physical_plan/json.rs | 43 ++--- .../core/src/datasource/physical_plan/mod.rs | 14 +- .../datasource/physical_plan/parquet/mod.rs | 43 ++--- .../enforce_distribution.rs | 182 ++++++++---------- .../src/physical_optimizer/enforce_sorting.rs | 2 +- .../src/physical_optimizer/join_selection.rs | 17 +- .../physical_optimizer/output_requirements.rs | 16 +- .../physical_optimizer/pipeline_checker.rs | 12 +- .../replace_with_order_preserving_variants.rs | 2 +- datafusion/core/src/physical_planner.rs | 39 ++-- datafusion/core/src/test/mod.rs | 15 +- datafusion/core/src/test_util/mod.rs | 30 ++- datafusion/core/tests/custom_sources.rs | 14 +- .../provider_filter_pushdown.rs | 19 +- .../tests/custom_sources_cases/statistics.rs | 19 +- .../tests/user_defined/user_defined_plan.rs | 23 +-- datafusion/physical-expr/src/partitioning.rs | 38 ++-- .../physical-plan/src/aggregates/mod.rs | 42 ++-- datafusion/physical-plan/src/analyze.rs | 14 +- .../physical-plan/src/coalesce_batches.rs | 18 +- .../physical-plan/src/coalesce_partitions.rs | 15 +- datafusion/physical-plan/src/display.rs | 24 +-- datafusion/physical-plan/src/empty.rs | 17 +- datafusion/physical-plan/src/explain.rs | 14 +- datafusion/physical-plan/src/filter.rs | 19 +- datafusion/physical-plan/src/insert.rs | 14 +- .../physical-plan/src/joins/cross_join.rs | 35 ++-- .../physical-plan/src/joins/hash_join.rs | 57 +++--- .../src/joins/nested_loop_join.rs | 25 +-- .../src/joins/sort_merge_join.rs | 19 +- .../src/joins/symmetric_hash_join.rs | 20 +- datafusion/physical-plan/src/joins/utils.rs | 21 +- datafusion/physical-plan/src/lib.rs | 124 ++++-------- datafusion/physical-plan/src/limit.rs | 30 +-- datafusion/physical-plan/src/memory.rs | 18 +- .../physical-plan/src/placeholder_row.rs | 17 +- datafusion/physical-plan/src/projection.rs | 16 +- .../physical-plan/src/recursive_query.rs | 27 ++- .../physical-plan/src/repartition/mod.rs | 59 +++--- .../physical-plan/src/sorts/partial_sort.rs | 34 ++-- datafusion/physical-plan/src/sorts/sort.rs | 15 +- .../src/sorts/sort_preserving_merge.rs | 16 +- datafusion/physical-plan/src/streaming.rs | 11 +- datafusion/physical-plan/src/test/exec.rs | 86 ++++----- datafusion/physical-plan/src/union.rs | 29 ++- datafusion/physical-plan/src/unnest.rs | 22 +-- datafusion/physical-plan/src/values.rs | 19 +- .../src/windows/bounded_window_agg_exec.rs | 24 +-- datafusion/physical-plan/src/windows/mod.rs | 5 +- .../src/windows/window_agg_exec.rs | 34 ++-- datafusion/physical-plan/src/work_table.rs | 29 ++- 58 files changed, 672 insertions(+), 926 deletions(-) diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 06d1ea4d46b9..da572b6d591f 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -34,13 +34,13 @@ use crate::{ use datafusion::common::plan_datafusion_err; use datafusion::datasource::listing::ListingTableUrl; use datafusion::error::{DataFusionError, Result}; +use datafusion::logical_expr::dml::CopyTo; use datafusion::logical_expr::{CreateExternalTable, DdlStatement, LogicalPlan}; use datafusion::physical_plan::{collect, execute_stream}; use datafusion::prelude::SessionContext; -use datafusion::sql::{parser::DFParser, sqlparser::dialect::dialect_from_str}; +use datafusion::sql::parser::{DFParser, Statement}; +use datafusion::sql::sqlparser::dialect::dialect_from_str; -use datafusion::logical_expr::dml::CopyTo; -use datafusion::sql::parser::Statement; use rustyline::error::ReadlineError; use rustyline::Editor; use tokio::signal; @@ -231,7 +231,7 @@ async fn exec_and_print( let df = ctx.execute_logical_plan(plan).await?; let physical_plan = df.create_physical_plan().await?; - if physical_plan.unbounded_output().is_unbounded() { + if physical_plan.execution_mode().is_unbounded() { let stream = execute_stream(physical_plan, task_ctx.clone())?; print_options.print_stream(stream, now).await?; } else { @@ -305,10 +305,9 @@ mod tests { use std::str::FromStr; use super::*; - use datafusion::common::plan_err; - use datafusion_common::{ - file_options::StatementOptions, FileType, FileTypeWriterOptions, - }; + + use datafusion::common::{plan_err, FileType, FileTypeWriterOptions}; + use datafusion_common::file_options::StatementOptions; async fn create_external_table_test(location: &str, sql: &str) -> Result<()> { let ctx = SessionContext::new(); diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 5ed1dfe05a14..9516dc570d6d 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -31,7 +31,7 @@ use datafusion::execution::context::{SessionState, TaskContext}; use datafusion::physical_plan::memory::MemoryStream; use datafusion::physical_plan::{ project_schema, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, - PlanPropertiesCache, SendableRecordBatchStream, + Partitioning, PlanPropertiesCache, SendableRecordBatchStream, }; use datafusion::prelude::*; use datafusion_expr::{Expr, LogicalPlanBuilder}; @@ -209,17 +209,11 @@ impl CustomExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; + self.cache = self + .cache + .with_partitioning(Partitioning::UnknownPartitioning(1)) + .with_exec_mode(ExecutionMode::Bounded); - // Output Partitioning - let output_partitioning = - datafusion::physical_plan::Partitioning::UnknownPartitioning(1); - new_cache = new_cache.with_partitioning(output_partitioning); - - // Execution Mode - new_cache = new_cache.with_exec_mode(ExecutionMode::Bounded); - - self.cache = new_cache; self } } diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 784b76fc3cae..cf7abc7e2314 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -28,10 +28,8 @@ use crate::arrow::record_batch::RecordBatch; use crate::arrow::util::pretty; use crate::datasource::{provider_as_source, MemTable, TableProvider}; use crate::error::Result; -use crate::execution::{ - context::{SessionState, TaskContext}, - FunctionRegistry, -}; +use crate::execution::context::{SessionState, TaskContext}; +use crate::execution::FunctionRegistry; use crate::logical_expr::utils::find_window_exprs; use crate::logical_expr::{ col, Expr, JoinType, LogicalPlan, LogicalPlanBuilder, Partitioning, TableType, @@ -40,6 +38,7 @@ use crate::physical_plan::{ collect, collect_partitioned, execute_stream, execute_stream_partitioned, ExecutionPlan, SendableRecordBatchStream, }; +use crate::prelude::SessionContext; use arrow::array::{Array, ArrayRef, Int64Array, StringArray}; use arrow::compute::{cast, concat}; @@ -58,7 +57,6 @@ use datafusion_expr::{ TableProviderFilterPushDown, UNNAMED_TABLE, }; -use crate::prelude::SessionContext; use async_trait::async_trait; /// Contains options that control how data is diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 1548d8160829..1a27f9315b34 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,6 +20,7 @@ use std::any::Any; use std::sync::Arc; +use super::FileGroupPartitioner; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, FileScanConfig, @@ -42,8 +43,6 @@ use futures::StreamExt; use itertools::Itertools; use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; -use super::FileGroupPartitioner; - /// Execution plan for scanning Arrow data source #[derive(Debug, Clone)] #[allow(dead_code)] @@ -89,14 +88,11 @@ impl ArrowExec { &self.projected_output_ordering, ); - // Output Partitioning - let output_partitioning = self.output_partitioning_helper(); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new( + eq_properties, + self.output_partitioning_helper(), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ); self } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 4b2521c81609..fb2cd627a1da 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -24,14 +24,13 @@ use super::FileScanConfig; use crate::error::Result; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, - Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, SendableRecordBatchStream, Statistics, }; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] @@ -73,16 +72,13 @@ impl AvroExec { self.schema(), &self.projected_output_ordering, ); + let n_partitions = self.base_config.file_groups.len(); - // Output Partitioning - let output_partitioning = - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(n_partitions), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ); self } } @@ -168,6 +164,7 @@ mod private { 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}; diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 831b2f40e93e..55cf62507788 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -32,18 +32,17 @@ 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, Partitioning, SendableRecordBatchStream, - Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, SendableRecordBatchStream, Statistics, }; 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::{ExecutionMode, PlanPropertiesCache}; use bytes::{Buf, Bytes}; -use datafusion_common::config::ConfigOptions; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; @@ -128,14 +127,11 @@ impl CsvExec { &self.projected_output_ordering, ); - // Output Partitioning - let output_partitioning = self.output_partitioning_helper(); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new( + eq_properties, + self.output_partitioning_helper(), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ); self } diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 9cb58e7032db..d21a1cd9fb21 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -28,9 +28,8 @@ use std::task::{Context, Poll}; use std::time::Instant; use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::{ - FileMeta, FileScanConfig, PartitionColumnProjector, -}; +use crate::datasource::physical_plan::file_scan_config::PartitionColumnProjector; +use crate::datasource::physical_plan::{FileMeta, FileScanConfig}; use crate::error::Result; use crate::physical_plan::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, Time, diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 5f3724cf2192..231b48f28d96 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -32,8 +32,8 @@ 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, Partitioning, SendableRecordBatchStream, - Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, SendableRecordBatchStream, Statistics, }; use arrow::json::ReaderBuilder; @@ -42,10 +42,8 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use futures::{ready, StreamExt, TryStreamExt}; -use object_store::{self, GetOptions}; -use object_store::{GetResultPayload, ObjectStore}; +use object_store::{self, GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; use tokio::task::JoinSet; @@ -97,14 +95,11 @@ impl NdJsonExec { &self.projected_output_ordering, ); - // Output Partitioning - let output_partitioning = self.output_partitioning_helper(); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new( + eq_properties, + self.output_partitioning_helper(), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ); self } @@ -378,11 +373,10 @@ pub async fn plan_to_json( #[cfg(test)] mod tests { - use arrow::array::Array; - use arrow::datatypes::{Field, SchemaBuilder}; - use futures::StreamExt; - use object_store::local::LocalFileSystem; + use std::fs; + use std::path::Path; + use super::*; use crate::assert_batches_eq; use crate::dataframe::DataFrameWriteOptions; use crate::datasource::file_format::file_compression_type::FileTypeExt; @@ -390,20 +384,23 @@ mod tests { use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; use crate::execution::context::SessionState; - use crate::prelude::NdJsonReadOptions; - use crate::prelude::*; + use crate::prelude::{ + CsvReadOptions, NdJsonReadOptions, SessionConfig, SessionContext, + }; use crate::test::partitioned_file_groups; + + use arrow::array::Array; + use arrow::datatypes::{Field, SchemaBuilder}; use datafusion_common::cast::{as_int32_array, as_int64_array, as_string_array}; use datafusion_common::FileType; + + use futures::StreamExt; use object_store::chunked::ChunkedStore; + use object_store::local::LocalFileSystem; use rstest::*; - use std::fs; - use std::path::Path; use tempfile::TempDir; use url::Url; - use super::*; - const TEST_DATA_BASE: &str = "tests/data"; async fn prepare_store( diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 31a45674b6bf..500fe4cb97f4 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -27,7 +27,6 @@ mod json; #[cfg(feature = "parquet")] pub mod parquet; pub use file_groups::FileGroupPartitioner; -use futures::StreamExt; pub(crate) use self::csv::plan_to_csv; pub use self::csv::{CsvConfig, CsvExec, CsvOpener}; @@ -37,7 +36,6 @@ pub use self::parquet::{ParquetExec, ParquetFileMetrics, ParquetFileReaderFactor pub use arrow_file::ArrowExec; pub use avro::AvroExec; -use file_scan_config::PartitionColumnProjector; pub use file_scan_config::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, }; @@ -72,9 +70,9 @@ use datafusion_common::{file_options::FileTypeWriterOptions, plan_err}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; +use futures::StreamExt; use log::debug; -use object_store::ObjectMeta; -use object_store::{path::Path, GetOptions, GetRange, ObjectStore}; +use object_store::{path::Path, GetOptions, GetRange, ObjectMeta, ObjectStore}; /// The base configurations to provide when creating a physical plan for /// writing to any given file format. @@ -589,6 +587,9 @@ async fn find_first_newline( #[cfg(test)] mod tests { + use super::*; + use crate::physical_plan::{DefaultDisplay, VerboseDisplay}; + use arrow_array::cast::AsArray; use arrow_array::types::{Float32Type, Float64Type, UInt32Type}; use arrow_array::{ @@ -596,11 +597,8 @@ mod tests { UInt64Array, }; use arrow_schema::Field; - use chrono::Utc; - - use crate::physical_plan::{DefaultDisplay, VerboseDisplay}; - use super::*; + use chrono::Utc; #[test] fn schema_mapping_map_batch() { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index d12fb5fafbf7..6499aa085219 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -38,15 +38,14 @@ use crate::{ physical_optimizer::pruning::PruningPredicate, physical_plan::{ metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, - Statistics, + DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, SendableRecordBatchStream, Statistics, }, }; use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use bytes::Bytes; use futures::future::BoxFuture; @@ -64,7 +63,7 @@ use parquet::schema::types::ColumnDescriptor; use tokio::task::JoinSet; mod metrics; -pub mod page_filter; +mod page_filter; mod row_filter; mod row_groups; mod statistics; @@ -273,14 +272,11 @@ impl ParquetExec { &self.projected_output_ordering, ); - // Output Partitioning - let output_partitioning = self.output_partitioning_helper(); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new( + eq_properties, + self.output_partitioning_helper(), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ); self } @@ -776,6 +772,8 @@ pub(crate) fn parquet_to_arrow_decimal_type( #[cfg(test)] mod tests { // See also `parquet_exec` integration test + use std::fs::{self, File}; + use std::io::Write; use super::*; use crate::dataframe::DataFrameWriteOptions; @@ -793,28 +791,25 @@ mod tests { datasource::file_format::{parquet::ParquetFormat, FileFormat}, physical_plan::collect, }; - use arrow::array::{ArrayRef, Int32Array}; - use arrow::datatypes::Schema; - use arrow::record_batch::RecordBatch; - use arrow::{ - array::{Int64Array, Int8Array, StringArray}, - datatypes::{DataType, Field, SchemaBuilder}, + + use arrow::array::{ + ArrayRef, Date64Array, Int32Array, Int64Array, Int8Array, StringArray, + StructArray, }; - use arrow_array::{Date64Array, StructArray}; + use arrow::datatypes::{DataType, Field, Schema, SchemaBuilder}; + use arrow::record_batch::RecordBatch; use arrow_schema::Fields; - use chrono::{TimeZone, Utc}; - use datafusion_common::{assert_contains, ToDFSchema}; - use datafusion_common::{FileType, GetExt, ScalarValue}; + use datafusion_common::{assert_contains, FileType, GetExt, ScalarValue, ToDFSchema}; use datafusion_expr::{col, lit, when, Expr}; use datafusion_physical_expr::create_physical_expr; use datafusion_physical_expr::execution_props::ExecutionProps; + + use chrono::{TimeZone, Utc}; use futures::StreamExt; use object_store::local::LocalFileSystem; use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; - use std::fs::{self, File}; - use std::io::Write; use tempfile::TempDir; use url::Url; diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index de77f53e49b0..c7bfe4742bdf 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -426,31 +426,27 @@ where let join_key_pairs = extract_join_keys(on); let eq_properties = join_plan.plan.equivalence_properties(); - if let Some(( + let ( JoinKeyPairs { left_keys, right_keys, }, - new_positions, - )) = try_reorder(join_key_pairs.clone(), parent_required, eq_properties) - { - if !new_positions.is_empty() { + positions, + ) = try_reorder(join_key_pairs, parent_required, eq_properties); + + if let Some(positions) = positions { + if !positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); let new_sort_options = (0..sort_options.len()) - .map(|idx| sort_options[new_positions[idx]]) + .map(|idx| sort_options[positions[idx]]) .collect(); join_plan.plan = join_constructor((new_join_on, new_sort_options))?; } - let mut requirements = join_plan; - requirements.children[0].data = left_keys; - requirements.children[1].data = right_keys; - Ok(requirements) - } else { - let mut requirements = join_plan; - requirements.children[0].data = join_key_pairs.left_keys; - requirements.children[1].data = join_key_pairs.right_keys; - Ok(requirements) } + let mut requirements = join_plan; + requirements.children[0].data = left_keys; + requirements.children[1].data = right_keys; + Ok(requirements) } fn reorder_aggregate_keys( @@ -604,32 +600,28 @@ pub(crate) fn reorder_join_keys_to_inputs( }) = plan_any.downcast_ref::() { if matches!(mode, PartitionMode::Partitioned) { - let join_key_pairs = extract_join_keys(on); - if let Some(( - JoinKeyPairs { - left_keys, - right_keys, - }, - new_positions, - )) = reorder_current_join_keys( - join_key_pairs, + let (join_keys, positions) = reorder_current_join_keys( + extract_join_keys(on), Some(left.output_partitioning()), Some(right.output_partitioning()), left.equivalence_properties(), right.equivalence_properties(), - ) { - if !new_positions.is_empty() { - let new_join_on = new_join_conditions(&left_keys, &right_keys); - return Ok(Arc::new(HashJoinExec::try_new( - left.clone(), - right.clone(), - new_join_on, - filter.clone(), - join_type, - PartitionMode::Partitioned, - *null_equals_null, - )?)); - } + ); + if positions.map_or(false, |idxs| !idxs.is_empty()) { + let JoinKeyPairs { + left_keys, + right_keys, + } = join_keys; + let new_join_on = new_join_conditions(&left_keys, &right_keys); + return Ok(Arc::new(HashJoinExec::try_new( + left.clone(), + right.clone(), + new_join_on, + filter.clone(), + join_type, + PartitionMode::Partitioned, + *null_equals_null, + )?)); } } } else if let Some(SortMergeJoinExec { @@ -643,24 +635,22 @@ pub(crate) fn reorder_join_keys_to_inputs( .. }) = plan_any.downcast_ref::() { - let join_key_pairs = extract_join_keys(on); - if let Some(( - JoinKeyPairs { - left_keys, - right_keys, - }, - new_positions, - )) = reorder_current_join_keys( - join_key_pairs, + let (join_keys, positions) = reorder_current_join_keys( + extract_join_keys(on), Some(left.output_partitioning()), Some(right.output_partitioning()), left.equivalence_properties(), right.equivalence_properties(), - ) { - if !new_positions.is_empty() { + ); + if let Some(positions) = positions { + if !positions.is_empty() { + let JoinKeyPairs { + left_keys, + right_keys, + } = join_keys; let new_join_on = new_join_conditions(&left_keys, &right_keys); let new_sort_options = (0..sort_options.len()) - .map(|idx| sort_options[new_positions[idx]]) + .map(|idx| sort_options[positions[idx]]) .collect(); return SortMergeJoinExec::try_new( left.clone(), @@ -685,24 +675,24 @@ fn reorder_current_join_keys( right_partition: Option<&Partitioning>, left_equivalence_properties: &EquivalenceProperties, right_equivalence_properties: &EquivalenceProperties, -) -> Option<(JoinKeyPairs, Vec)> { +) -> (JoinKeyPairs, Option>) { match (left_partition, right_partition) { (Some(Partitioning::Hash(left_exprs, _)), _) => { - try_reorder(join_keys.clone(), left_exprs, left_equivalence_properties) - .or_else(|| { - reorder_current_join_keys( - join_keys, - None, - right_partition, - left_equivalence_properties, - right_equivalence_properties, - ) - }) + match try_reorder(join_keys, left_exprs, left_equivalence_properties) { + (join_keys, None) => reorder_current_join_keys( + join_keys, + None, + right_partition, + left_equivalence_properties, + right_equivalence_properties, + ), + result => result, + } } (_, Some(Partitioning::Hash(right_exprs, _))) => { try_reorder(join_keys, right_exprs, right_equivalence_properties) } - _ => None, + _ => (join_keys, None), } } @@ -710,66 +700,65 @@ fn try_reorder( join_keys: JoinKeyPairs, expected: &[Arc], equivalence_properties: &EquivalenceProperties, -) -> Option<(JoinKeyPairs, Vec)> { +) -> (JoinKeyPairs, Option>) { let eq_groups = equivalence_properties.eq_group(); let mut normalized_expected = vec![]; let mut normalized_left_keys = vec![]; let mut normalized_right_keys = vec![]; if join_keys.left_keys.len() != expected.len() { - return None; + return (join_keys, None); } if physical_exprs_equal(expected, &join_keys.left_keys) || physical_exprs_equal(expected, &join_keys.right_keys) { - return Some((join_keys, vec![])); + return (join_keys, Some(vec![])); } else if !equivalence_properties.eq_group().is_empty() { normalized_expected = expected .iter() .map(|e| eq_groups.normalize_expr(e.clone())) .collect(); - assert_eq!(normalized_expected.len(), expected.len()); normalized_left_keys = join_keys .left_keys .iter() .map(|e| eq_groups.normalize_expr(e.clone())) .collect(); - assert_eq!(join_keys.left_keys.len(), normalized_left_keys.len()); normalized_right_keys = join_keys .right_keys .iter() .map(|e| eq_groups.normalize_expr(e.clone())) .collect(); - assert_eq!(join_keys.right_keys.len(), normalized_right_keys.len()); if physical_exprs_equal(&normalized_expected, &normalized_left_keys) || physical_exprs_equal(&normalized_expected, &normalized_right_keys) { - return Some((join_keys, vec![])); + return (join_keys, Some(vec![])); } } - let new_positions = expected_expr_positions(&join_keys.left_keys, expected) + let Some(positions) = expected_expr_positions(&join_keys.left_keys, expected) .or_else(|| expected_expr_positions(&join_keys.right_keys, expected)) .or_else(|| expected_expr_positions(&normalized_left_keys, &normalized_expected)) .or_else(|| { expected_expr_positions(&normalized_right_keys, &normalized_expected) - }); - - new_positions.map(|positions| { - let mut new_left_keys = vec![]; - let mut new_right_keys = vec![]; - for pos in positions.iter() { - new_left_keys.push(join_keys.left_keys[*pos].clone()); - new_right_keys.push(join_keys.right_keys[*pos].clone()); - } - let pairs = JoinKeyPairs { - left_keys: new_left_keys, - right_keys: new_right_keys, - }; - (pairs, positions) - }) + }) + else { + return (join_keys, None); + }; + + let mut new_left_keys = vec![]; + let mut new_right_keys = vec![]; + for pos in positions.iter() { + new_left_keys.push(join_keys.left_keys[*pos].clone()); + new_right_keys.push(join_keys.right_keys[*pos].clone()); + } + let pairs = JoinKeyPairs { + left_keys: new_left_keys, + right_keys: new_right_keys, + }; + + (pairs, Some(positions)) } /// Return the expected expressions positions. @@ -882,12 +871,11 @@ fn add_hash_on_top( return Ok(input); } + let dist = Distribution::HashPartitioned(hash_exprs); let satisfied = input .plan .output_partitioning() - .satisfy(Distribution::HashPartitioned(hash_exprs.clone()), || { - input.plan.equivalence_properties().clone() - }); + .satisfy(&dist, input.plan.equivalence_properties()); // Add hash repartitioning when: // - The hash distribution requirement is not satisfied, or @@ -900,7 +888,7 @@ fn add_hash_on_top( // requirements. // - Usage of order preserving variants is not desirable (per the flag // `config.optimizer.prefer_existing_sort`). - let partitioning = Partitioning::Hash(hash_exprs, n_target); + let partitioning = dist.create_partitioning(n_target); let repartition = RepartitionExec::try_new(input.plan.clone(), partitioning)? .with_preserve_order(); let plan = Arc::new(repartition) as _; @@ -1076,7 +1064,7 @@ fn ensure_distribution( let enable_round_robin = config.optimizer.enable_round_robin_repartition; let repartition_file_scans = config.optimizer.repartition_file_scans; let batch_size = config.execution.batch_size; - let is_unbounded = dist_context.plan.unbounded_output().is_unbounded(); + let is_unbounded = dist_context.plan.execution_mode().is_unbounded(); // Use order preserving variants either of the conditions true // - it is desired according to config // - when plan is unbounded @@ -1374,17 +1362,11 @@ pub(crate) mod tests { } fn with_cache(mut self) -> Self { - // Equivalence Properties - let eq_properties = self.input.equivalence_properties().clone(); - - // Output Partitioning - let output_partitioning = self.input.output_partitioning().clone(); - - // Execution Mode - let exec_mode = self.input.unbounded_output(); - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new( + self.input.equivalence_properties().clone(), // Equivalence Properties + self.input.output_partitioning().clone(), // Output Partitioning + self.input.execution_mode(), // Execution Mode + ); self } } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index ea83e2b6b38f..b459c86518b6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -207,7 +207,7 @@ fn replace_with_partial_sort( let plan_any = plan.as_any(); if let Some(sort_plan) = plan_any.downcast_ref::() { let child = sort_plan.children()[0].clone(); - if !child.unbounded_output().is_unbounded() { + if !child.execution_mode().is_unbounded() { return Ok(plan); } diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 8e3d14543b3f..f74732305372 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -38,8 +38,7 @@ use crate::physical_plan::ExecutionPlan; use arrow_schema::Schema; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{internal_err, JoinSide}; -use datafusion_common::{DataFusionError, JoinType}; +use datafusion_common::{internal_err, DataFusionError, JoinSide, JoinType}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::sort_properties::SortProperties; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -465,8 +464,8 @@ fn hash_join_convert_symmetric_subrule( ) -> Result> { // Check if the current plan node is a HashJoinExec. if let Some(hash_join) = input.as_any().downcast_ref::() { - let left_unbounded = hash_join.left.unbounded_output().is_unbounded(); - let right_unbounded = hash_join.right.unbounded_output().is_unbounded(); + let left_unbounded = hash_join.left.execution_mode().is_unbounded(); + let right_unbounded = hash_join.right.execution_mode().is_unbounded(); // Process only if both left and right sides are unbounded. if left_unbounded && right_unbounded { // Determine the partition mode based on configuration. @@ -595,10 +594,8 @@ fn hash_join_swap_subrule( _config_options: &ConfigOptions, ) -> Result> { if let Some(hash_join) = input.as_any().downcast_ref::() { - let left_unbounded = hash_join.left.unbounded_output().is_unbounded(); - let right_unbounded = hash_join.right.unbounded_output().is_unbounded(); - if left_unbounded - && !right_unbounded + if hash_join.left.execution_mode().is_unbounded() + && !hash_join.right.execution_mode().is_unbounded() && matches!( *hash_join.join_type(), JoinType::Inner @@ -1779,12 +1776,12 @@ mod hash_join_tests { assert_eq!( ( t.case.as_str(), - if left.unbounded_output().is_unbounded() { + if left.execution_mode().is_unbounded() { SourceType::Unbounded } else { SourceType::Bounded }, - if right.unbounded_output().is_unbounded() { + if right.execution_mode().is_unbounded() { SourceType::Unbounded } else { SourceType::Bounded diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 52a1afdbd1ba..5ddba79515ee 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -114,17 +114,11 @@ impl OutputRequirementExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - let eq_properties = self.input.equivalence_properties().clone(); - - // Output Partitioning - let output_partitioning = self.input.output_partitioning().clone(); - - // Execution Mode - let exec_mode = self.input.unbounded_output(); - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new( + self.input.equivalence_properties().clone(), // Equivalence Properties + self.input.output_partitioning().clone(), // Output Partitioning + self.input.execution_mode(), // Execution Mode + ); self } } diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 9f694a5c7695..9a7afcb6409d 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -28,7 +28,7 @@ use crate::physical_plan::ExecutionPlan; use datafusion_common::config::OptimizerOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{plan_datafusion_err, plan_err, DataFusionError}; +use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; @@ -50,9 +50,7 @@ impl PhysicalOptimizerRule for PipelineChecker { plan: Arc, config: &ConfigOptions, ) -> Result> { - let plan = - plan.transform_up(&|p| check_finiteness_requirements(p, &config.optimizer))?; - Ok(plan) + plan.transform_up(&|p| check_finiteness_requirements(p, &config.optimizer)) } fn name(&self) -> &str { @@ -78,11 +76,11 @@ pub fn check_finiteness_requirements( the 'allow_symmetric_joins_without_pruning' configuration flag"); } } - if !input.unbounded_output().pipeline_friendly() { - Err(plan_datafusion_err!( + if !input.execution_mode().pipeline_friendly() { + plan_err!( "Cannot execute pipeline breaking queries, operator: {:?}", input - )) + ) } else { Ok(Transformed::No(input)) } 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 5fc24205f5da..8825feb45e98 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 @@ -241,7 +241,7 @@ pub(crate) fn replace_with_order_preserving_variants( // For unbounded cases, we replace with the order-preserving variant in any // case, as doing so helps fix the pipeline. Also replace if config allows. let use_order_preserving_variant = config.optimizer.prefer_existing_sort - || !requirements.plan.unbounded_output().pipeline_friendly(); + || !requirements.plan.execution_mode().pipeline_friendly(); // Create an alternate plan with order-preserving variants: let mut alternate_plan = plan_with_order_preserving_variants( diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 165c8818f403..03e601eacf95 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1976,31 +1976,35 @@ fn tuple_err(value: (Result, Result)) -> Result<(T, R)> { #[cfg(test)] mod tests { + use std::any::Any; + use std::collections::HashMap; + use std::convert::TryFrom; + use std::fmt::{self, Debug}; + use std::ops::{BitAnd, Not}; + use super::*; use crate::datasource::file_format::options::CsvReadOptions; use crate::datasource::MemTable; - use crate::physical_plan::{expressions, DisplayFormatType, Partitioning}; - use crate::physical_plan::{DisplayAs, SendableRecordBatchStream}; + use crate::physical_plan::{ + expressions, DisplayAs, DisplayFormatType, ExecutionMode, Partitioning, + PlanPropertiesCache, SendableRecordBatchStream, + }; use crate::physical_planner::PhysicalPlanner; use crate::prelude::{SessionConfig, SessionContext}; use crate::test_util::{scan_empty, scan_empty_with_partitions}; + use arrow::array::{ArrayRef, DictionaryArray, Int32Array}; use arrow::datatypes::{DataType, Field, Int32Type, SchemaRef}; use arrow::record_batch::RecordBatch; - use datafusion_common::{assert_contains, TableReference}; - use datafusion_common::{DFField, DFSchema, DFSchemaRef}; + use datafusion_common::{ + assert_contains, DFField, DFSchema, DFSchemaRef, TableReference, + }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{ col, lit, sum, Extension, GroupingSet, LogicalPlanBuilder, UserDefinedLogicalNodeCore, }; - use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; - use fmt::Debug; - use std::collections::HashMap; - use std::convert::TryFrom; - use std::ops::{BitAnd, Not}; - use std::{any::Any, fmt}; fn make_session_state() -> SessionState { let runtime = Arc::new(RuntimeEnv::default()); @@ -2572,16 +2576,13 @@ mod tests { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); - - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); - - // Execution Mode - new_cache = new_cache.with_exec_mode(ExecutionMode::Bounded); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 80560fd8d107..277901ff9915 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -369,6 +369,7 @@ pub struct StatisticsExec { schema: Arc, cache: PlanPropertiesCache, } + impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { assert_eq!( @@ -385,15 +386,13 @@ impl StatisticsExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(2)); - - // Execution Mode - new_cache = new_cache.with_exec_mode(ExecutionMode::Bounded); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(2)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 8f5d7164e136..bb016f93c351 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -29,17 +29,16 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use tempfile::TempDir; - use crate::dataframe::DataFrame; use crate::datasource::provider::TableProviderFactory; +use crate::datasource::stream::{StreamConfig, StreamTable}; use crate::datasource::{empty::EmptyTable, provider_as_source, TableProvider}; use crate::error::Result; use crate::execution::context::{SessionState, TaskContext}; use crate::logical_expr::{LogicalPlanBuilder, UNNAMED_TABLE}; use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, }; use crate::prelude::{CsvReadOptions, SessionContext}; @@ -50,16 +49,13 @@ use datafusion_expr::{CreateExternalTable, Expr, TableType}; use async_trait::async_trait; use futures::Stream; +use tempfile::TempDir; // backwards compatibility #[cfg(feature = "parquet")] pub use datafusion_common::test_util::parquet_test_data; pub use datafusion_common::test_util::{arrow_test_data, get_data_dir}; -use crate::datasource::stream::{StreamConfig, StreamTable}; -pub use datafusion_common::{assert_batches_eq, assert_batches_sorted_eq}; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; - /// Scan an empty data source, mainly used in tests pub fn scan_empty( name: Option<&str>, @@ -253,21 +249,17 @@ impl UnboundedExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - - // Output Partitioning - new_cache = new_cache - .with_partitioning(Partitioning::UnknownPartitioning(self.partitions)); - - // Execution Mode - let exec_mode = if self.batch_produce.is_none() { + let mode = if self.batch_produce.is_none() { ExecutionMode::Unbounded } else { ExecutionMode::Bounded }; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(self.partitions)) + // Execution Mode + .with_exec_mode(mode); self } } diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 005b02a14054..a167258ee1d5 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -86,15 +86,13 @@ impl CustomExecutionPlan { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index 522c18a4d654..9423f0170c7e 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -27,15 +27,14 @@ use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, - Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanPropertiesCache, SendableRecordBatchStream, Statistics, }; use datafusion::prelude::*; use datafusion::scalar::ScalarValue; use datafusion_common::cast::as_primitive_array; use datafusion_common::{internal_err, not_impl_err, DataFusionError}; use datafusion_expr::expr::{BinaryExpr, Cast}; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use async_trait::async_trait; @@ -68,15 +67,13 @@ impl CustomPlan { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 41915c1fd49e..315c7cb6dd26 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -26,14 +26,13 @@ use datafusion::{ error::Result, logical_expr::Expr, physical_plan::{ - ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, + Partitioning, PlanPropertiesCache, SendableRecordBatchStream, Statistics, }, prelude::SessionContext, scalar::ScalarValue, }; use datafusion_common::{project_schema, stats::Precision}; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use async_trait::async_trait; @@ -63,15 +62,13 @@ impl StatisticsValidation { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(2)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(2)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index f6488711b9f7..947376dfb6a0 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -70,7 +70,7 @@ use arrow::{ }; use datafusion::{ common::cast::{as_int64_array, as_string_array}, - common::{internal_err, DFSchemaRef}, + common::{arrow_datafusion_err, internal_err, DFSchemaRef}, error::{DataFusionError, Result}, execution::{ context::{QueryPlanner, SessionState, TaskContext}, @@ -82,14 +82,13 @@ use datafusion::{ }, optimizer::{optimize_children, OptimizerConfig, OptimizerRule}, physical_plan::{ - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, + Partitioning, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + Statistics, }, physical_planner::{DefaultPhysicalPlanner, ExtensionPlanner, PhysicalPlanner}, prelude::{SessionConfig, SessionContext}, }; -use datafusion_common::arrow_datafusion_err; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; use async_trait::async_trait; use futures::{Stream, StreamExt}; @@ -422,15 +421,13 @@ impl TopKExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 301f12e9aa2e..a4a246daf676 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -142,12 +142,12 @@ impl Partitioning { } } - /// Returns true when the guarantees made by this [[Partitioning]] are sufficient to - /// satisfy the partitioning scheme mandated by the `required` [[Distribution]] - pub fn satisfy EquivalenceProperties>( + /// Returns true when the guarantees made by this [`Partitioning`] are sufficient to + /// satisfy the partitioning scheme mandated by the `required` [`Distribution`]. + pub fn satisfy( &self, - required: Distribution, - eq_properties: F, + required: &Distribution, + eq_properties: &EquivalenceProperties, ) -> bool { match required { Distribution::UnspecifiedDistribution => true, @@ -159,11 +159,10 @@ impl Partitioning { // then we need to have the partition count and hash functions validation. Partitioning::Hash(partition_exprs, _) => { let fast_match = - physical_exprs_equal(&required_exprs, partition_exprs); + physical_exprs_equal(required_exprs, partition_exprs); // If the required exprs do not match, need to leverage the eq_properties provided by the child // and normalize both exprs based on the equivalent groups. if !fast_match { - let eq_properties = eq_properties(); let eq_groups = eq_properties.eq_group(); if !eq_groups.is_empty() { let normalized_required_exprs = required_exprs @@ -222,14 +221,14 @@ pub enum Distribution { impl Distribution { /// Creates a `Partitioning` that satisfies this `Distribution` - pub fn create_partitioning(&self, partition_count: usize) -> Partitioning { + pub fn create_partitioning(self, partition_count: usize) -> Partitioning { match self { Distribution::UnspecifiedDistribution => { Partitioning::UnknownPartitioning(partition_count) } Distribution::SinglePartition => Partitioning::UnknownPartitioning(1), Distribution::HashPartitioned(expr) => { - Partitioning::Hash(expr.clone(), partition_count) + Partitioning::Hash(expr, partition_count) } } } @@ -273,24 +272,15 @@ mod tests { let round_robin_partition = Partitioning::RoundRobinBatch(10); let hash_partition1 = Partitioning::Hash(partition_exprs1, 10); let hash_partition2 = Partitioning::Hash(partition_exprs2, 10); + let eq_properties = EquivalenceProperties::new(schema); for distribution in distribution_types { let result = ( - single_partition.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) - }), - unspecified_partition.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) - }), - round_robin_partition.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) - }), - hash_partition1.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) - }), - hash_partition2.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) - }), + single_partition.satisfy(&distribution, &eq_properties), + unspecified_partition.satisfy(&distribution, &eq_properties), + round_robin_partition.satisfy(&distribution, &eq_properties), + hash_partition1.satisfy(&distribution, &eq_properties), + hash_partition2.satisfy(&distribution, &eq_properties), ); match distribution { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a686f8f462d6..ea3fc3a737b2 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -506,22 +506,20 @@ impl AggregateExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Construct equivalence properties: let eq_properties = self .input .equivalence_properties() .project(&self.projection_mapping, self.schema()); - // Output partitioning + // Get output partitioning: let mut output_partitioning = self.input.output_partitioning().clone(); if self.mode.is_first_stage() { // First stage aggregation will not change the output partitioning, // but needs to respect aliases (e.g. mapping in the GROUP BY // expression). let input_eq_properties = self.input.equivalence_properties(); - // First stage Aggregation will not change the output partitioning but need to respect the Alias - let input_partition = self.input.output_partitioning(); - if let Partitioning::Hash(exprs, part) = input_partition { + if let Partitioning::Hash(exprs, part) = output_partitioning { let normalized_exprs = exprs .iter() .map(|expr| { @@ -532,24 +530,22 @@ impl AggregateExec { }) }) .collect(); - output_partitioning = Partitioning::Hash(normalized_exprs, *part); + output_partitioning = Partitioning::Hash(normalized_exprs, part); } } - // Unbounded Output - let mut unbounded_output = self.input.unbounded_output(); - if self.input.unbounded_output() == ExecutionMode::Unbounded + // Determine execution mode: + let mut exec_mode = self.input.execution_mode(); + if exec_mode == ExecutionMode::Unbounded && self.input_order_mode == InputOrderMode::Linear { - // Cannot run without breaking pipeline. - unbounded_output = ExecutionMode::PipelineBreaking; + // Cannot run without breaking the pipeline + exec_mode = ExecutionMode::PipelineBreaking; } - self.cache = PlanPropertiesCache::new( - eq_properties, - output_partitioning, - unbounded_output, - ); + self.cache = + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self } @@ -1631,15 +1627,13 @@ mod tests { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 7d4bdebfe8a0..99b3add2acd6 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -82,15 +82,13 @@ impl AnalyzeExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(self.input.execution_mode()); - // Execution Mode - let exec_mode = self.input.unbounded_output(); - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 8d25f7df1d8f..e01060f3784d 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -75,18 +75,14 @@ impl CoalesceBatchesExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - let eq_properties = self.input.equivalence_properties().clone(); + // The coalesce batches operator does not make any changes to the + // partitioning of its input. + self.cache = PlanPropertiesCache::new( + self.input.equivalence_properties().clone(), // Equivalence Properties + self.input.output_partitioning().clone(), // Output Partitioning + self.input.execution_mode(), // Execution Mode + ); - // Output Partitioning - // The coalesce batches operator does not make any changes to the partitioning of its input - let output_partitioning = self.input.output_partitioning().clone(); - - // Execution Mode - let exec_mode = self.input.unbounded_output(); - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); self } } diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 4815aaa2d4f0..255e996bd122 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -59,19 +59,16 @@ impl CoalescePartitionsExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Coalescing partitions loses existing orderings: let mut eq_properties = self.input.equivalence_properties().clone(); - // Coalesce partitions loses existing orderings. eq_properties.clear_orderings(); - // Output Partitioning - let output_partitioning = Partitioning::UnknownPartitioning(1); + self.cache = PlanPropertiesCache::new( + eq_properties, // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + self.input.execution_mode(), // Execution Mode + ); - // Execution Mode - let exec_mode = self.input.unbounded_output(); - - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); self } } diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index fe5be52bc001..d4afca6a1acf 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -303,10 +303,7 @@ impl GraphvizVisitor<'_, '_> { impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { type Error = fmt::Error; - fn pre_visit( - &mut self, - plan: &dyn ExecutionPlan, - ) -> datafusion_common::Result { + fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { let id = self.graphviz_builder.next_id(); struct Wrapper<'a>(&'a dyn ExecutionPlan, DisplayFormatType); @@ -468,11 +465,11 @@ mod tests { use std::fmt::Write; use std::sync::Arc; - use datafusion_common::DataFusionError; - + use super::DisplayableExecutionPlan; use crate::{DisplayAs, ExecutionPlan, PlanPropertiesCache}; - use super::DisplayableExecutionPlan; + use datafusion_common::{DataFusionError, Result, Statistics}; + use datafusion_execution::{SendableRecordBatchStream, TaskContext}; #[derive(Debug, Clone, Copy)] enum TestStatsExecPlan { @@ -511,28 +508,25 @@ mod tests { fn with_new_children( self: Arc, _: Vec>, - ) -> datafusion_common::Result> { + ) -> Result> { unimplemented!() } fn execute( &self, _: usize, - _: Arc, - ) -> datafusion_common::Result - { + _: Arc, + ) -> Result { todo!() } - fn statistics(&self) -> datafusion_common::Result { + fn statistics(&self) -> Result { match self { Self::Panic => panic!("expected panic"), Self::Error => { Err(DataFusionError::Internal("expected error".to_string())) } - Self::Ok => Ok(datafusion_common::Statistics::new_unknown( - self.schema().as_ref(), - )), + Self::Ok => Ok(Statistics::new_unknown(self.schema().as_ref())), } } } diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index dfd1e9d12904..d91395825135 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -58,8 +58,7 @@ impl EmptyExec { /// Create a new EmptyExec with specified partition number pub fn with_partitions(mut self, partitions: usize) -> Self { self.partitions = partitions; - // Changing partitions may invalidate output partitioning. - // Update it also. + // Changing partitions may invalidate output partitioning, so update it: let output_partitioning = self.output_partitioning_helper(); self.cache = self.cache.with_partitioning(output_partitioning); self @@ -74,17 +73,15 @@ impl EmptyExec { } fn with_cache(mut self) -> Self { - // Output Partitioning let output_partitioning = self.output_partitioning_helper(); - let mut new_cache = self.cache; - new_cache = new_cache.with_partitioning(output_partitioning); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(output_partitioning) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 1f7beb632501..935b37c168da 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -73,15 +73,13 @@ impl ExplainExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 9e0b644cd4fb..cc8fdcbcd0cd 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -116,9 +116,9 @@ impl FilterExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - let stats = self.statistics().unwrap(); // Combine the equal predicates with the input equivalence properties + // to construct the equivalence properties: + let stats = self.statistics().unwrap(); let mut eq_properties = self.input.equivalence_properties().clone(); let (equal_pairs, _) = collect_columns_from_predicate(&self.predicate); for (lhs, rhs) in equal_pairs { @@ -126,21 +126,20 @@ impl FilterExec { let rhs_expr = Arc::new(rhs.clone()) as _; eq_properties.add_equal_conditions(&lhs_expr, &rhs_expr) } - // Add the columns that have only one value (singleton) after filtering to constants. + // Add the columns that have only one viable value (singleton) after + // filtering to constants. let constants = collect_columns(self.predicate()) .into_iter() .filter(|column| stats.column_statistics[column.index()].is_singleton()) .map(|column| Arc::new(column) as _); eq_properties = eq_properties.add_constants(constants); - // Output Partitioning - let output_partitioning = self.input.output_partitioning().clone(); - - // Execution Mode - let exec_mode = self.input.unbounded_output(); + self.cache = PlanPropertiesCache::new( + eq_properties, + self.input.output_partitioning().clone(), // Output Partitioning + self.input.execution_mode(), // Execution Mode + ); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); self } } diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index bdc1c3bc40e3..b20e8cac7926 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -177,15 +177,13 @@ impl FileSinkExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(self.input.execution_mode()); - // Execution Mode - let exec_mode = self.input.unbounded_output(); - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 0f4fea1cab96..8b12b02b4667 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -23,13 +23,14 @@ use std::{any::Any, sync::Arc, task::Poll}; use super::utils::{ adjust_right_output_partitioning, BuildProbeJoinMetrics, OnceAsync, OnceFut, }; +use crate::coalesce_batches::concat_batches; +use crate::coalesce_partitions::CoalescePartitionsExec; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, - ColumnStatistics, DisplayFormatType, Distribution, ExecutionPlan, - PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, + ExecutionPlan, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + Statistics, }; -use crate::{DisplayAs, ExecutionMode}; use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; @@ -99,8 +100,9 @@ impl CrossJoinExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - // TODO: Check equivalence properties of crossjoin, for some cases it may preserve ordering. + // Calculate equivalence properties + // TODO: Check equivalence properties of cross join, it may preserve + // ordering in some cases. let eq_properties = join_equivalence_properties( self.left.equivalence_properties().clone(), self.right.equivalence_properties().clone(), @@ -111,25 +113,22 @@ impl CrossJoinExec { &[], ); - // Output Partitioning - // TODO optimize CrossJoin implementation to generate M * N partitions - let left_columns_len = self.left.schema().fields.len(); + // Get output partitioning: + // TODO: Optimize the cross join implementation to generate M * N + // partitions. let output_partitioning = adjust_right_output_partitioning( - self.right.output_partitioning().clone(), - left_columns_len, + self.right.output_partitioning(), + self.left.schema().fields.len(), ); - // Execution Mode - let left_unbounded = self.left.unbounded_output(); - let right_unbounded = self.right.unbounded_output(); - let exec_mode = match (left_unbounded, right_unbounded) { + // Determine the execution mode: + let mode = match (self.left.execution_mode(), self.right.execution_mode()) { (ExecutionMode::Bounded, ExecutionMode::Bounded) => ExecutionMode::Bounded, - // If any of the inputs is unbounded, cross join break pipeline. + // If any of the inputs is unbounded, cross join breaks the pipeline. (_, _) => ExecutionMode::PipelineBreaking, }; - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 8fb36b728ecb..7afe252f21ee 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -23,29 +23,26 @@ use std::sync::Arc; use std::task::Poll; use std::{any::Any, usize, vec}; -use crate::joins::utils::{ - adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices, - get_final_indices_from_bit_map, need_produce_result_in_final, JoinHashMap, - JoinHashMapOffset, JoinHashMapType, +use super::{ + utils::{OnceAsync, OnceFut}, + PartitionMode, }; use crate::{ coalesce_partitions::CoalescePartitionsExec, - exec_mode_flatten, + exec_mode_flatten, handle_state, hash_utils::create_hashes, joins::utils::{ - adjust_right_output_partitioning, build_join_schema, check_join_is_valid, - estimate_join_statistics, partitioned_join_output_partitioning, - BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, StatefulStreamResult, + adjust_indices_by_join_type, adjust_right_output_partitioning, + apply_join_filter_to_indices, build_batch_from_indices, build_join_schema, + check_join_is_valid, estimate_join_statistics, get_final_indices_from_bit_map, + need_produce_result_in_final, partitioned_join_output_partitioning, + BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinHashMap, JoinHashMapOffset, + JoinHashMapType, JoinOn, StatefulStreamResult, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, -}; -use crate::{handle_state, DisplayAs}; - -use super::{ - utils::{OnceAsync, OnceFut}, - PartitionMode, + DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, + Partitioning, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + Statistics, }; use arrow::array::{ @@ -299,6 +296,7 @@ pub struct HashJoinExec { /// Otherwise, rows that have `null`s in the join columns will not be /// matched and thus will not appear in the output. pub null_equals_null: bool, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -405,7 +403,7 @@ impl HashJoinExec { let left = &self.left; let right = &self.right; let schema = self.schema(); - // Equivalence properties + // Calculate equivalence properties: let eq_properties = join_equivalence_properties( left.equivalence_properties().clone(), right.equivalence_properties().clone(), @@ -416,12 +414,12 @@ impl HashJoinExec { &self.on, ); - // Output partitioning + // Get output partitioning: let left_columns_len = left.schema().fields.len(); let output_partitioning = match self.mode { PartitionMode::CollectLeft => match self.join_type { JoinType::Inner | JoinType::Right => adjust_right_output_partitioning( - right.output_partitioning().clone(), + right.output_partitioning(), left_columns_len, ), JoinType::RightSemi | JoinType::RightAnti => { @@ -436,8 +434,8 @@ impl HashJoinExec { }, PartitionMode::Partitioned => partitioned_join_output_partitioning( self.join_type, - left.output_partitioning().clone(), - right.output_partitioning().clone(), + left.output_partitioning(), + right.output_partitioning(), left_columns_len, ), PartitionMode::Auto => Partitioning::UnknownPartitioning( @@ -445,13 +443,11 @@ impl HashJoinExec { ), }; - // Unbounded output - let left_unbounded = left.unbounded_output().is_unbounded(); - let right_unbounded = right.unbounded_output().is_unbounded(); - // If left is unbounded, or right is unbounded with JoinType::Right, - // JoinType::Full, JoinType::RightAnti types. - let breaking = left_unbounded - || (right_unbounded + // Determine execution mode by checking whether this join is pipeline + // breaking. This happens when the left side is unbounded, or the right + // side is unbounded with `Right`, `Full` or `RightAnti` join types. + let pipeline_breaking = left.execution_mode().is_unbounded() + || (right.execution_mode().is_unbounded() && matches!( self.join_type, JoinType::Left @@ -460,14 +456,13 @@ impl HashJoinExec { | JoinType::LeftSemi )); - let exec_mode = if breaking { + let mode = if pipeline_breaking { ExecutionMode::PipelineBreaking } else { exec_mode_flatten([left, right]) }; - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 724735f28077..d4c3051e551b 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -34,8 +34,8 @@ use crate::joins::utils::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - exec_mode_safe_flatten, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, - PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + exec_mode_flatten, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, + ExecutionPlan, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, }; use arrow::array::{ @@ -92,6 +92,7 @@ pub struct NestedLoopJoinExec { column_indices: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -144,7 +145,7 @@ impl NestedLoopJoinExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Calculate equivalence properties: let eq_properties = join_equivalence_properties( self.left.equivalence_properties().clone(), self.right.equivalence_properties().clone(), @@ -156,24 +157,26 @@ impl NestedLoopJoinExec { &[], ); - // Output Partitioning - // the partition of output is determined by the rule of `required_input_distribution` + // Get output partitioning, which is determined by the rule of + // `required_input_distribution`: let output_partitioning = if self.join_type == JoinType::Full { self.left.output_partitioning().clone() } else { partitioned_join_output_partitioning( self.join_type, - self.left.output_partitioning().clone(), - self.right.output_partitioning().clone(), + self.left.output_partitioning(), + self.right.output_partitioning(), self.left.schema().fields.len(), ) }; - // Execution Mode - let exec_mode = exec_mode_safe_flatten([&self.left, &self.right]); + // Determine execution mode: + let mut mode = exec_mode_flatten([&self.left, &self.right]); + if mode.is_unbounded() { + mode = ExecutionMode::PipelineBreaking; + } - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 2f9729351a31..0d7cd995a5a2 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -43,8 +43,7 @@ use crate::{ }; use arrow::array::*; -use arrow::compute; -use arrow::compute::{concat_batches, take, SortOptions}; +use arrow::compute::{self, concat_batches, take, SortOptions}; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; @@ -84,6 +83,7 @@ pub struct SortMergeJoinExec { pub sort_options: Vec, /// If null_equals_null is true, null == null else null != null pub null_equals_null: bool, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -202,7 +202,7 @@ impl SortMergeJoinExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Calculate equivalence properties: let eq_properties = join_equivalence_properties( self.left.equivalence_properties().clone(), self.right.equivalence_properties().clone(), @@ -213,20 +213,19 @@ impl SortMergeJoinExec { self.on(), ); - // Output Partitioning + // Get output partitioning: let left_columns_len = self.left.schema().fields.len(); let output_partitioning = partitioned_join_output_partitioning( self.join_type, - self.left.output_partitioning().clone(), - self.right.output_partitioning().clone(), + self.left.output_partitioning(), + self.right.output_partitioning(), left_columns_len, ); - // Execution Mode - let exec_mode = exec_mode_flatten([&self.left, &self.right]); + // Determine execution mode: + let mode = exec_mode_flatten([&self.left, &self.right]); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index f636fc197168..5fd89e98a58f 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -71,9 +71,9 @@ use datafusion_execution::TaskContext; use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; +use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use ahash::RandomState; -use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use futures::Stream; use hashbrown::HashSet; use parking_lot::Mutex; @@ -191,6 +191,7 @@ pub struct SymmetricHashJoinExec { pub(crate) right_sort_exprs: Option>, /// Partition Mode mode: StreamJoinPartitionMode, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -253,7 +254,7 @@ impl SymmetricHashJoinExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Calculate equivalence properties: let eq_properties = join_equivalence_properties( self.left.equivalence_properties().clone(), self.right.equivalence_properties().clone(), @@ -265,20 +266,19 @@ impl SymmetricHashJoinExec { self.on(), ); - // Output Partitioning + // Get output partitioning: let left_columns_len = self.left.schema().fields.len(); let output_partitioning = partitioned_join_output_partitioning( self.join_type, - self.left.output_partitioning().clone(), - self.right.output_partitioning().clone(), + self.left.output_partitioning(), + self.right.output_partitioning(), left_columns_len, ); - // Execution Mode - let exec_mode = exec_mode_flatten([&self.left, &self.right]); + // Determine execution mode: + let mode = exec_mode_flatten([&self.left, &self.right]); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } @@ -1328,11 +1328,11 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}; + use datafusion_common::ScalarValue; use datafusion_execution::config::SessionConfig; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{binary, col, lit, Column}; - use datafusion_common::ScalarValue; use once_cell::sync::Lazy; use rstest::*; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index e6e3f83fd7e8..3dac0107d3ef 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -432,15 +432,15 @@ fn check_join_set_is_valid( /// Calculate the OutputPartitioning for Partitioned Join pub fn partitioned_join_output_partitioning( join_type: JoinType, - left_partitioning: Partitioning, - right_partitioning: Partitioning, + left_partitioning: &Partitioning, + right_partitioning: &Partitioning, left_columns_len: usize, ) -> Partitioning { match join_type { JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - left_partitioning + left_partitioning.clone() } - JoinType::RightSemi | JoinType::RightAnti => right_partitioning, + JoinType::RightSemi | JoinType::RightAnti => right_partitioning.clone(), JoinType::Right => { adjust_right_output_partitioning(right_partitioning, left_columns_len) } @@ -452,21 +452,18 @@ pub fn partitioned_join_output_partitioning( /// Adjust the right out partitioning to new Column Index pub fn adjust_right_output_partitioning( - right_partitioning: Partitioning, + right_partitioning: &Partitioning, left_columns_len: usize, ) -> Partitioning { match right_partitioning { - Partitioning::RoundRobinBatch(size) => Partitioning::RoundRobinBatch(size), - Partitioning::UnknownPartitioning(size) => { - Partitioning::UnknownPartitioning(size) - } Partitioning::Hash(exprs, size) => { let new_exprs = exprs - .into_iter() - .map(|expr| add_offset_to_expr(expr, left_columns_len)) + .iter() + .map(|expr| add_offset_to_expr(expr.clone(), left_columns_len)) .collect(); - Partitioning::Hash(new_exprs, size) + Partitioning::Hash(new_exprs, *size) } + result => result.clone(), } } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index e7e451586fbc..52bfb0063d40 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -29,6 +29,7 @@ use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; +use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::Transformed; use datafusion_common::utils::DataPtr; use datafusion_common::{DataFusionError, Result}; @@ -41,6 +42,7 @@ use datafusion_physical_expr::{ use futures::stream::TryStreamExt; use tokio::task::JoinSet; +mod ordering; mod topk; mod visitor; @@ -58,7 +60,6 @@ pub mod joins; pub mod limit; pub mod memory; pub mod metrics; -mod ordering; pub mod placeholder_row; pub mod projection; pub mod recursive_query; @@ -80,7 +81,6 @@ pub use crate::ordering::InputOrderMode; pub use crate::topk::TopK; pub use crate::visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; -use datafusion_common::config::ConfigOptions; pub use datafusion_common::hash_utils; pub use datafusion_common::utils::project_schema; pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; @@ -136,7 +136,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Specifies whether this plan generates an infinite stream of records. /// If the plan does not support pipelining, but its input(s) are /// infinite, returns an error to indicate this. - fn unbounded_output(&self) -> ExecutionMode { + fn execution_mode(&self) -> ExecutionMode { self.cache().exec_mode } @@ -450,33 +450,21 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } } -/// Describes the execution mode of each operator's resulting stream. -/// -/// This enum defines the possible execution modes: -/// -/// - `Bounded`: Represents a mode where execution is limited or constrained within certain bounds. -/// In this mode, the process operates within defined limits or boundaries. -/// -/// - `Unbounded`: Indicates a mode where execution is not limited by any specific bounds. -/// Processes or tasks in this mode can operate without constraints or limitations. -/// -/// - `PipelineBreaking`: Denotes a mode where execution can potentially break pipeline constraints. -/// This mode may disrupt established pipelines or processes that rely on sequential operation. -/// -/// This enum can be used to specify the behavior or characteristics of a process or task -/// in various execution scenarios. +/// Describes the execution mode of an operator's resulting stream with respect +/// to its size and behavior. There are three possible execution modes: `Bounded`, +/// `Unbounded` and `PipelineBreaking`. #[derive(Clone, Copy, PartialEq, Debug)] pub enum ExecutionMode { /// Represents the mode where generated stream is bounded, e.g. finite. Bounded, /// Represents the mode where generated stream is unbounded, e.g. infinite. - /// Operator can generate streaming results with bounded memory. - /// In this mode, execution can still continue successfully. + /// Even though the operator generates an unbounded stream of results, it + /// works with bounded memory and execution can still continue successfully. Unbounded, - /// Represents the mode, where input stream to the operator is unbounded. However, - /// operator cannot generate streaming results from streaming inputs. In this case, - /// execution mode will be pipeline breaking. e.g. operator requires unbounded memory - /// to generate its result. + /// Represents the mode where some of the operator's input stream(s) are + /// unbounded; however, the operator cannot generate streaming results from + /// these streaming inputs. In this case, the execution mode will be pipeline + /// breaking, e.g. the operator requires unbounded memory to generate results. PipelineBreaking, } @@ -486,33 +474,31 @@ impl ExecutionMode { matches!(self, ExecutionMode::Unbounded) } - /// Check whether the execution is pipeline friendly. If so, operator can execute - /// safely. + /// Check whether the execution is pipeline friendly. If so, operator can + /// execute safely. pub fn pipeline_friendly(&self) -> bool { matches!(self, ExecutionMode::Bounded | ExecutionMode::Unbounded) } } -fn exec_mode_flatten_helper(modes: &[ExecutionMode]) -> ExecutionMode { +/// Conservatively "combines" execution modes of a given collection of operators. +fn exec_mode_flatten<'a>( + children: impl IntoIterator>, +) -> ExecutionMode { let mut result = ExecutionMode::Bounded; - for mode in modes { + for mode in children.into_iter().map(|child| child.execution_mode()) { match (mode, result) { (ExecutionMode::PipelineBreaking, _) | (_, ExecutionMode::PipelineBreaking) => { - // If any of the modes is `InExecutable`. result is `PipelineBreaking` - // Return immediately + // If any of the modes is `PipelineBreaking`, so is the result: return ExecutionMode::PipelineBreaking; } - ( - ExecutionMode::Unbounded, - ExecutionMode::Bounded | ExecutionMode::Unbounded, - ) - | (ExecutionMode::Bounded, ExecutionMode::Unbounded) => { - // Unbounded mode eats up bounded mode. + (ExecutionMode::Unbounded, _) | (_, ExecutionMode::Unbounded) => { + // Unbounded mode eats up bounded mode: result = ExecutionMode::Unbounded; } (ExecutionMode::Bounded, ExecutionMode::Bounded) => { - // When both modes are bounded, result is bounded + // When both modes are bounded, so is the result: result = ExecutionMode::Bounded; } } @@ -520,63 +506,20 @@ fn exec_mode_flatten_helper(modes: &[ExecutionMode]) -> ExecutionMode { result } -/// Constructs execution mode of the operator using its children. -/// This util assumes operator applied is pipeline friendly. For operators -/// not pipeline friendly, `exec_mode_safe_flatten` should be used. -fn exec_mode_flatten<'a>( - children: impl IntoIterator>, -) -> ExecutionMode { - let modes = children - .into_iter() - .map(|child| child.unbounded_output()) - .collect::>(); - exec_mode_flatten_helper(&modes) -} - -fn exec_mode_safe_flatten_helper(modes: &[ExecutionMode]) -> ExecutionMode { - let mut result = ExecutionMode::Bounded; - for mode in modes { - match (mode, result) { - (ExecutionMode::Bounded, ExecutionMode::Bounded) => { - // When both modes are bounded, result is bounded - result = ExecutionMode::Bounded; - } - (_, _) => { - // If any of the modes is `InExecutable` or `Unbounded`. result is `PipelineBreaking` - // Return immediately - return ExecutionMode::PipelineBreaking; - } - } - } - result -} - -/// Constructs execution mode of the operator using its children. -/// This util assumes operator applied is not pipeline friendly. For operators -/// that pipeline friendly, `exec_mode_flatten` should be used. -fn exec_mode_safe_flatten<'a>( - children: impl IntoIterator>, -) -> ExecutionMode { - let modes = children - .into_iter() - .map(|child| child.unbounded_output()) - .collect::>(); - exec_mode_safe_flatten_helper(&modes) -} - /// Represents a cache for plan properties used in query optimization. /// /// This struct holds various properties useful for the query planning, which are used /// during optimization and execution phases. #[derive(Debug, Clone)] pub struct PlanPropertiesCache { - /// Stores Equivalence Properties of the [`ExecutionPlan`]. See [`EquivalenceProperties`] + /// Stores the [`EquivalenceProperties`] of the [`ExecutionPlan`]. pub eq_properties: EquivalenceProperties, - /// Stores Output Partitioning of the [`ExecutionPlan`]. See [`Partitioning`] + /// Stores the output [`Partitioning`] of the [`ExecutionPlan`]. pub partitioning: Partitioning, - /// Stores Execution Mode of the [`ExecutionPlan`]. See [`ExecutionMode`] + /// Stores the [`ExecutionMode`] of the [`ExecutionPlan`]. pub exec_mode: ExecutionMode, - /// Stores output ordering of the [`ExecutionPlan`]. `None` represents, no ordering. + /// Stores output ordering of the [`ExecutionPlan`]. A `None` value represents + /// no ordering. output_ordering: Option, } @@ -599,7 +542,7 @@ impl PlanPropertiesCache { /// Construct a default `PlanPropertiesCache`, for a given schema. pub fn new_default(schema: SchemaRef) -> PlanPropertiesCache { - // Defaults are most restrictive possible values. + // Default values are the most restrictive possible values. let eq_properties = EquivalenceProperties::new(schema); // Please note that this default is not safe, and should be overwritten. let partitioning = Partitioning::UnknownPartitioning(0); @@ -613,21 +556,22 @@ impl PlanPropertiesCache { } } - /// Overwrite partitioning with its new value + /// Overwrite output partitioning with its new value. pub fn with_partitioning(mut self, partitioning: Partitioning) -> Self { self.partitioning = partitioning; self } - /// Overwrite Execution Mode with its new value + /// Overwrite the execution Mode with its new value. pub fn with_exec_mode(mut self, exec_mode: ExecutionMode) -> Self { self.exec_mode = exec_mode; self } - /// Overwrite Equivalence Properties with its new value + /// Overwrite equivalence properties with its new value. pub fn with_eq_properties(mut self, eq_properties: EquivalenceProperties) -> Self { - // Changing equivalence properties, changes output ordering also. Make sure to overwrite it. + // Changing equivalence properties also changes output ordering, so + // make sure to overwrite it: self.output_ordering = eq_properties.oeq_class().output_ordering(); self.eq_properties = eq_properties; self diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 8b7313436469..59559e84fd75 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -84,17 +84,12 @@ impl GlobalLimitExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - let eq_properties = self.input.equivalence_properties().clone(); - - // Output Partitioning - let output_partitioning = Partitioning::UnknownPartitioning(1); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; + self.cache = PlanPropertiesCache::new( + self.input.equivalence_properties().clone(), // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); self } } @@ -302,17 +297,12 @@ impl LocalLimitExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - let eq_properties = self.input.equivalence_properties().clone(); - - // Output Partitioning - let output_partitioning = self.input.output_partitioning().clone(); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; + self.cache = PlanPropertiesCache::new( + self.input.equivalence_properties().clone(), // Equivalence Properties + self.input.output_partitioning().clone(), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); self } } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 3a994f41a0be..206d22e72226 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -192,7 +192,7 @@ impl MemoryExec { pub fn with_sort_information(mut self, sort_information: Vec) -> Self { self.sort_information = sort_information; - // With updated sort information, we need to update equivalence properties also. + // We need to update equivalence properties when updating sort information. let eq_properties = self.equivalent_properties_helper(); self.cache = self.cache.with_eq_properties(eq_properties); self.with_cache() @@ -207,18 +207,12 @@ impl MemoryExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - let eq_properties = self.equivalent_properties_helper(); - - // Output Partitioning - let output_partitioning = - Partitioning::UnknownPartitioning(self.partitions.len()); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; + self.cache = PlanPropertiesCache::new( + self.equivalent_properties_helper(), // Equivalence Properties + Partitioning::UnknownPartitioning(self.partitions.len()), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); self } } diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index a35d259f153a..a9b27cb13fe1 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -60,7 +60,7 @@ impl PlaceholderRowExec { /// Create a new PlaceholderRowExecPlaceholderRowExec with specified partition number pub fn with_partitions(mut self, partitions: usize) -> Self { self.partitions = partitions; - // When changing partitions, output partitions should change also. + // Update output partitioning when updating partitions: let output_partitioning = self.output_partitioning_helper(); self.cache = self.cache.with_partitioning(output_partitioning); self @@ -94,16 +94,14 @@ impl PlaceholderRowExec { } fn with_cache(mut self) -> Self { + // Get output partitioning: let output_partitioning = self.output_partitioning_helper(); - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(output_partitioning); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); + self.cache = self + .cache + .with_partitioning(output_partitioning) + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } @@ -180,8 +178,7 @@ impl ExecutionPlan for PlaceholderRowExec { #[cfg(test)] mod tests { use super::*; - use crate::with_new_children_if_necessary; - use crate::{common, test}; + use crate::{common, test, with_new_children_if_necessary}; #[test] fn with_new_children() -> Result<()> { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index eb45e701544e..68592972498e 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -61,6 +61,7 @@ pub struct ProjectionExec { projection_mapping: ProjectionMapping, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -119,13 +120,12 @@ impl ProjectionExec { fn with_cache(mut self) -> Self { let input = &self.input; - // Equivalence properties + // Calculate equivalence properties: let input_eq_properties = input.equivalence_properties(); let eq_properties = input_eq_properties.project(&self.projection_mapping, self.schema.clone()); - // output partitioning - // Output partition need to respect the alias + // Calculate output partitioning, which needs to respect aliases: let input_partition = input.output_partitioning(); let output_partitioning = if let Partitioning::Hash(exprs, part) = input_partition { @@ -144,16 +144,12 @@ impl ProjectionExec { input_partition.clone() }; - // unbounded output - let unbounded_output = input.unbounded_output(); - - // Construct cache - let cache = PlanPropertiesCache::new( + self.cache = PlanPropertiesCache::new( eq_properties, output_partitioning, - unbounded_output, + input.execution_mode(), ); - self.cache = cache; + self } } diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index e49c94d6c646..97a626c25116 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -21,13 +21,12 @@ use std::any::Any; use std::sync::Arc; use std::task::{Context, Poll}; -use super::metrics::BaselineMetrics; use super::{ - metrics::{ExecutionPlanMetricsSet, MetricsSet}, + metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, work_table::{WorkTable, WorkTableExec}, - ExecutionMode, SendableRecordBatchStream, Statistics, + PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use super::{PlanPropertiesCache, RecordBatchStream}; +use crate::{DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -37,8 +36,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::Partitioning; use futures::{ready, Stream, StreamExt}; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan}; - /// Recursive query execution plan. /// /// This plan has two components: a base part (the static term) and @@ -68,6 +65,7 @@ pub struct RecursiveQueryExec { is_distinct: bool, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -97,15 +95,11 @@ impl RecursiveQueryExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); + self.cache = self + .cache + .with_partitioning(Partitioning::UnknownPartitioning(1)) + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } @@ -144,12 +138,13 @@ impl ExecutionPlan for RecursiveQueryExec { self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(RecursiveQueryExec::try_new( + RecursiveQueryExec::try_new( self.name.clone(), children[0].clone(), children[1].clone(), self.is_distinct, - )?)) + ) + .map(|e| Arc::new(e) as _) } fn execute( diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 54b5a5404d24..6d2835df05f0 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -24,36 +24,34 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::{any::Any, vec}; -use arrow::array::{ArrayRef, UInt64Builder}; -use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; -use futures::stream::Stream; -use futures::{FutureExt, StreamExt}; -use hashbrown::HashMap; -use log::trace; -use parking_lot::Mutex; -use tokio::task::JoinHandle; - -use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Result}; -use datafusion_execution::memory_pool::MemoryConsumer; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalExpr; - +use super::common::{AbortOnDropMany, AbortOnDropSingle, SharedMemoryReservation}; +use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream}; use crate::common::transpose; use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; -use crate::repartition::distributor_channels::{channels, partition_aware_channels}; +use crate::repartition::distributor_channels::{ + channels, partition_aware_channels, DistributionReceiver, DistributionSender, +}; use crate::sorts::streaming_merge; use crate::{ DisplayFormatType, ExecutionPlan, Partitioning, PlanPropertiesCache, Statistics, }; -use super::common::{AbortOnDropMany, AbortOnDropSingle, SharedMemoryReservation}; -use super::expressions::PhysicalSortExpr; -use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream}; +use arrow::array::{ArrayRef, UInt64Builder}; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Result}; +use datafusion_execution::memory_pool::MemoryConsumer; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use self::distributor_channels::{DistributionReceiver, DistributionSender}; +use futures::stream::Stream; +use futures::{FutureExt, StreamExt}; +use hashbrown::HashMap; +use log::trace; +use parking_lot::Mutex; +use tokio::task::JoinHandle; mod distributor_channels; @@ -297,19 +295,16 @@ impl BatchPartitioner { pub struct RepartitionExec { /// Input execution plan input: Arc, - /// Partitioning scheme to use partitioning: Partitioning, - /// Inner state that is initialized when the first output stream is created. state: Arc>, - /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Boolean flag to decide whether to preserve ordering. If true means /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. preserve_order: bool, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -625,19 +620,17 @@ impl RepartitionExec { fn with_cache(mut self) -> Self { // Equivalence Properties let mut eq_properties = self.input.equivalence_properties().clone(); - // If the ordering is lost, reset the ordering equivalence class. + // If the ordering is lost, reset the ordering equivalence class: if !self.maintains_input_order()[0] { eq_properties.clear_orderings(); } - // Output Partitioning - let output_partitioning = self.partitioning.clone(); - - // Execution Mode - let exec_mode = self.input.unbounded_output(); + self.cache = PlanPropertiesCache::new( + eq_properties, // Equivalence Properties + self.partitioning.clone(), // Output Partitioning + self.input.execution_mode(), // Execution Mode + ); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); self } diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index d59a3577acc0..99ead9f904a1 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -57,16 +57,6 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use arrow::compute::concat_batches; -use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; -use futures::{ready, Stream, StreamExt}; -use log::trace; - -use datafusion_common::utils::evaluate_partition_ranges; -use datafusion_common::Result; -use datafusion_execution::{RecordBatchStream, TaskContext}; - use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::sort::sort_batch; @@ -75,6 +65,16 @@ use crate::{ PlanPropertiesCache, SendableRecordBatchStream, Statistics, }; +use arrow::compute::concat_batches; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use datafusion_common::utils::evaluate_partition_ranges; +use datafusion_common::Result; +use datafusion_execution::{RecordBatchStream, TaskContext}; + +use futures::{ready, Stream, StreamExt}; +use log::trace; + /// Partial Sort execution plan. #[derive(Debug, Clone)] pub struct PartialSortExec { @@ -92,6 +92,7 @@ pub struct PartialSortExec { preserve_partitioning: bool, /// Fetch highest/lowest n results fetch: Option, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -161,26 +162,25 @@ impl PartialSortExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - // Reset the ordering equivalence class with the new ordering: + // Calculate equivalence properties; i.e. reset the ordering equivalence + // class with the new ordering: let eq_properties = self .input .equivalence_properties() .clone() .with_reorder(self.expr.to_vec()); - // Output Partitioning + // Get output partitioning: let output_partitioning = if self.preserve_partitioning { self.input.output_partitioning().clone() } else { Partitioning::UnknownPartitioning(1) }; - // Execution Mode - let exec_mode = self.input.unbounded_output(); + // Determine execution mode: + let mode = self.input.execution_mode(); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index e97c550cdcd4..55a3c9f068f1 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -676,6 +676,7 @@ pub struct SortExec { preserve_partitioning: bool, /// Fetch highest/lowest n results fetch: Option, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -767,31 +768,31 @@ impl SortExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - // Reset the ordering equivalence class with the new ordering: + // Calculate equivalence properties; i.e. reset the ordering equivalence + // class with the new ordering: let eq_properties = self .input .equivalence_properties() .clone() .with_reorder(self.expr.to_vec()); - // Output Partitioning + // Get output partitioning: let output_partitioning = if self.preserve_partitioning { self.input.output_partitioning().clone() } else { Partitioning::UnknownPartitioning(1) }; - // Execution Mode - let exec_mode = match self.input.unbounded_output() { + // Determine execution mode: + let mode = match self.input.execution_mode() { ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { ExecutionMode::PipelineBreaking } ExecutionMode::Bounded => ExecutionMode::Bounded, }; - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); + self } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 6a0ab8cb5748..eadd2d0711fe 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -73,6 +73,7 @@ pub struct SortPreservingMergeExec { metrics: ExecutionPlanMetricsSet, /// Optional number of rows to fetch. Stops producing rows after this fetch fetch: Option, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -111,17 +112,12 @@ impl SortPreservingMergeExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties - let eq_properties = self.input.equivalence_properties().clone(); - - // Output Partitioning - let output_partitioning = Partitioning::UnknownPartitioning(1); - - // Execution Mode - let exec_mode = self.input.unbounded_output(); + self.cache = PlanPropertiesCache::new( + self.input.equivalence_properties().clone(), // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + self.input.execution_mode(), // Execution Mode + ); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); self } } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index dbb7805261a7..60b372446805 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -122,25 +122,24 @@ impl StreamingTableExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Calculate equivalence properties: let eq_properties = EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ); - // Output Partitioning + // Get output partitioning: let output_partitioning = Partitioning::UnknownPartitioning(self.partitions.len()); - // Execution Mode - let exec_mode = if self.infinite { + // Determine execution mode: + let mode = if self.infinite { ExecutionMode::Unbounded } else { ExecutionMode::Bounded }; - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index cfbbd054bac2..77ff8d27157a 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -151,15 +151,13 @@ impl MockExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } @@ -310,16 +308,13 @@ impl BarrierExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache - .with_partitioning(Partitioning::UnknownPartitioning(self.data.len())); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(self.data.len())) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } @@ -422,15 +417,13 @@ impl ErrorExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(1)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } @@ -503,15 +496,13 @@ impl StatisticsExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(2)); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(2)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } @@ -607,16 +598,13 @@ impl BlockingExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache - .with_partitioning(Partitioning::UnknownPartitioning(self.n_partitions)); - - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(self.n_partitions)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } @@ -747,17 +735,15 @@ impl PanicExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning let num_partitions = self.batches_until_panics.len(); - new_cache = new_cache - .with_partitioning(Partitioning::UnknownPartitioning(num_partitions)); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(Partitioning::UnknownPartitioning(num_partitions)) + // Execution Mode + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 974ea630b160..42e5ce58edb0 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -90,6 +90,7 @@ pub struct UnionExec { inputs: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -112,7 +113,7 @@ impl UnionExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Calculate equivalence properties: // TODO: In some cases, we should be able to preserve some equivalence // classes and constants. Add support for such cases. let children_eqs = self @@ -150,8 +151,7 @@ impl UnionExec { // entries (implicitly) and return: eq_properties.add_new_orderings(meets); - // Output Partitioning - // Output the combination of all output partitions of the inputs if the Union is not partition aware + // Calculate output partitioning; i.e. sum output partitions of the inputs. let num_partitions = self .inputs .iter() @@ -159,11 +159,10 @@ impl UnionExec { .sum(); let output_partitioning = Partitioning::UnknownPartitioning(num_partitions); - // Execution Mode - let exec_mode = exec_mode_flatten(self.inputs.iter()); + // Determine execution mode: + let mode = exec_mode_flatten(self.inputs.iter()); - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } } @@ -317,6 +316,7 @@ pub struct InterleaveExec { inputs: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -345,17 +345,16 @@ impl InterleaveExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - - // Output Partitioning + // Get output partitioning: let output_partitioning = self.inputs[0].output_partitioning().clone(); - new_cache = new_cache.with_partitioning(output_partitioning); + // Determine execution mode: + let mode = exec_mode_flatten(self.inputs.iter()); - // Execution Mode - let exec_mode = exec_mode_flatten(self.inputs.iter()); - new_cache = new_cache.with_exec_mode(exec_mode); + self.cache = self + .cache + .with_partitioning(output_partitioning) + .with_exec_mode(mode); - self.cache = new_cache; self } } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 64c575831562..b9df57d84f81 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -19,6 +19,7 @@ //! type, conceptually is like joining each row with all the values in the list column. use std::{any::Any, sync::Arc}; +use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use super::{DisplayAs, PlanPropertiesCache}; use crate::{ expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, PhysicalExpr, @@ -26,15 +27,14 @@ use crate::{ }; use arrow::array::{ - Array, ArrayRef, ArrowPrimitiveType, FixedSizeListArray, LargeListArray, ListArray, - PrimitiveArray, + Array, ArrayRef, ArrowPrimitiveType, FixedSizeListArray, GenericListArray, + LargeListArray, ListArray, OffsetSizeTrait, PrimitiveArray, }; use arrow::compute::kernels; use arrow::datatypes::{ ArrowNativeType, DataType, Int32Type, Int64Type, Schema, SchemaRef, }; use arrow::record_batch::RecordBatch; -use arrow_array::{GenericListArray, OffsetSizeTrait}; use datafusion_common::{exec_err, DataFusionError, Result, UnnestOptions}; use datafusion_execution::TaskContext; @@ -42,8 +42,6 @@ use async_trait::async_trait; use futures::{Stream, StreamExt}; use log::trace; -use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; - /// Unnest the given column by joining the row with each value in the /// nested type. /// @@ -60,6 +58,7 @@ pub struct UnnestExec { options: UnnestOptions, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -84,14 +83,13 @@ impl UnnestExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(self.input.output_partitioning().clone()); - - // Execution Mode - new_cache = new_cache.with_exec_mode(self.input.unbounded_output()); + self.cache = self + .cache + // Output Partitioning + .with_partitioning(self.input.output_partitioning().clone()) + // Execution Mode + .with_exec_mode(self.input.execution_mode()); - self.cache = new_cache; self } } diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 6351ad8f5b10..7fc242099379 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -41,6 +41,7 @@ pub struct ValuesExec { schema: SchemaRef, /// The data data: Vec, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -127,15 +128,11 @@ impl ValuesExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); + self.cache = self + .cache + .with_partitioning(Partitioning::UnknownPartitioning(1)) + .with_exec_mode(ExecutionMode::Bounded); - // Execution Mode - let exec_mode = ExecutionMode::Bounded; - new_cache = new_cache.with_exec_mode(exec_mode); - - self.cache = new_cache; self } } @@ -172,10 +169,8 @@ impl ExecutionPlan for ValuesExec { self: Arc, _: Vec>, ) -> Result> { - Ok(Arc::new(ValuesExec::try_new_from_batches( - self.schema.clone(), - self.data.clone(), - )?)) + ValuesExec::try_new_from_batches(self.schema.clone(), self.data.clone()) + .map(|e| Arc::new(e) as _) } fn execute( 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 d3b9ff74b644..cb512302cb6f 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -89,6 +89,7 @@ pub struct BoundedWindowAggExec { // `ordered_partition_by_indices` would be 0, 1. // See `get_ordered_partition_by_indices` for more details. ordered_partition_by_indices: Vec, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -183,7 +184,7 @@ impl BoundedWindowAggExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Calculate equivalence properties: let eq_properties = window_equivalence_properties(&self.schema, &self.input, &self.window_expr); @@ -192,14 +193,11 @@ impl BoundedWindowAggExec { // repartitioning. let output_partitioning = self.input.output_partitioning().clone(); - // unbounded output - let unbounded_output = self.input.unbounded_output(); - // Construct properties cache self.cache = PlanPropertiesCache::new( - eq_properties, - output_partitioning, - unbounded_output, + eq_properties, // Equivalence Properties + output_partitioning, // Output Partitioning + self.input.execution_mode(), // Execution Mode ); self } @@ -1117,21 +1115,23 @@ fn get_aggregate_result_out_column( #[cfg(test)] mod tests { + use std::sync::Arc; + use crate::common::collect; use crate::memory::MemoryExec; use crate::windows::{BoundedWindowAggExec, InputOrderMode}; use crate::{get_plan_string, ExecutionPlan}; + use arrow_array::RecordBatch; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{assert_batches_eq, Result, ScalarValue}; use datafusion_execution::config::SessionConfig; use datafusion_execution::TaskContext; use datafusion_expr::{WindowFrame, WindowFrameBound, WindowFrameUnits}; - use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::expressions::NthValue; - use datafusion_physical_expr::window::BuiltInWindowExpr; - use datafusion_physical_expr::window::BuiltInWindowFunctionExpr; - use std::sync::Arc; + use datafusion_physical_expr::expressions::{col, NthValue}; + use datafusion_physical_expr::window::{ + BuiltInWindowExpr, BuiltInWindowFunctionExpr, + }; // Tests NTH_VALUE(negative index) with memoize feature. // To be able to trigger memoize feature for NTH_VALUE we need to diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 4143b5c9f508..14e3d190d77c 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -48,11 +48,10 @@ mod bounded_window_agg_exec; mod window_agg_exec; pub use bounded_window_agg_exec::BoundedWindowAggExec; -pub use window_agg_exec::WindowAggExec; - pub use datafusion_physical_expr::window::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, }; +pub use window_agg_exec::WindowAggExec; /// Create a physical expression for window function pub fn create_window_expr( @@ -415,7 +414,7 @@ pub fn get_best_fitting_window( } else { return Ok(None); }; - let is_unbounded = input.unbounded_output().is_unbounded(); + let is_unbounded = input.execution_mode().is_unbounded(); if !is_unbounded && input_order_mode != InputOrderMode::Sorted { // Executor has bounded input and `input_order_mode` is not `InputOrderMode::Sorted` // in this case removing the sort is not helpful, return: diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index ddfce03e2026..247588c971a2 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -35,22 +35,18 @@ use crate::{ SendableRecordBatchStream, Statistics, WindowExpr, }; +use arrow::array::ArrayRef; use arrow::compute::{concat, concat_batches}; -use arrow::datatypes::SchemaBuilder; +use arrow::datatypes::{Schema, SchemaBuilder, SchemaRef}; use arrow::error::ArrowError; -use arrow::{ - array::ArrayRef, - datatypes::{Schema, SchemaRef}, - record_batch::RecordBatch, -}; +use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortRequirement; -use futures::stream::Stream; -use futures::{ready, StreamExt}; +use futures::{ready, Stream, StreamExt}; /// Window execution plan #[derive(Debug)] @@ -68,6 +64,7 @@ pub struct WindowAggExec { /// Partition by indices that defines preset for existing ordering // see `get_ordered_partition_by_indices` for more details. ordered_partition_by_indices: Vec, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -121,30 +118,25 @@ impl WindowAggExec { } fn with_cache(mut self) -> Self { - // Equivalence Properties + // Calculate equivalence properties: let eq_properties = window_equivalence_properties(&self.schema, &self.input, &self.window_expr); - // output partitioning - // because we can have repartitioning using the partition keys - // this would be either 1 or more than 1 depending on the presense of - // repartitioning + // Get output partitioning: + // Because we can have repartitioning using the partition keys this + // would be either 1 or more than 1 depending on the presense of repartitioning. let output_partitioning = self.input.output_partitioning().clone(); - // unbounded output - let unbounded_output = match self.input.unbounded_output() { + // Determine execution mode: + let mode = match self.input.execution_mode() { ExecutionMode::Bounded => ExecutionMode::Bounded, ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { ExecutionMode::PipelineBreaking } }; - // Construct properties cache - self.cache = PlanPropertiesCache::new( - eq_properties, - output_partitioning, - unbounded_output, - ); + // Construct properties cache: + self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); self } } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index 13bc98f1878b..33c611dd30d8 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -20,21 +20,20 @@ use std::any::Any; use std::sync::{Arc, Mutex}; -use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::Partitioning; - +use super::{ + metrics::{ExecutionPlanMetricsSet, MetricsSet}, + SendableRecordBatchStream, Statistics, +}; use crate::memory::MemoryStream; use crate::{ DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, PlanPropertiesCache, }; -use super::{ - metrics::{ExecutionPlanMetricsSet, MetricsSet}, - SendableRecordBatchStream, Statistics, -}; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::Partitioning; /// The name is from PostgreSQL's terminology. /// See @@ -85,6 +84,7 @@ pub struct WorkTableExec { work_table: Arc, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanPropertiesCache, } @@ -113,14 +113,11 @@ impl WorkTableExec { } fn with_cache(mut self) -> Self { - let mut new_cache = self.cache; - // Output Partitioning - new_cache = new_cache.with_partitioning(Partitioning::UnknownPartitioning(1)); - - // Execution Mode - new_cache = new_cache.with_exec_mode(ExecutionMode::Bounded); + self.cache = self + .cache + .with_partitioning(Partitioning::UnknownPartitioning(1)) + .with_exec_mode(ExecutionMode::Bounded); - self.cache = new_cache; self } } From 93f52824b9f2f6afdbf1b438969c7ae48c96e22d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 21 Feb 2024 17:56:03 +0300 Subject: [PATCH 04/15] Minor changes --- datafusion/physical-plan/src/joins/hash_join.rs | 2 +- datafusion/physical-plan/src/joins/nested_loop_join.rs | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 7afe252f21ee..3aae053151cd 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -445,7 +445,7 @@ impl HashJoinExec { // Determine execution mode by checking whether this join is pipeline // breaking. This happens when the left side is unbounded, or the right - // side is unbounded with `Right`, `Full` or `RightAnti` join types. + // side is unbounded with `Left`, `Full`, `LeftAnti` or `LeftSemi` join types. let pipeline_breaking = left.execution_mode().is_unbounded() || (right.execution_mode().is_unbounded() && matches!( diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index d4c3051e551b..1978338d2b6a 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -157,8 +157,7 @@ impl NestedLoopJoinExec { &[], ); - // Get output partitioning, which is determined by the rule of - // `required_input_distribution`: + // Get output partitioning, let output_partitioning = if self.join_type == JoinType::Full { self.left.output_partitioning().clone() } else { From 10000fb5bdb88a5e7a5cbfb81f3bd5d028bbeea7 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Thu, 22 Feb 2024 09:56:23 +0300 Subject: [PATCH 05/15] Delete docs.yaml --- .github/workflows/docs.yaml | 64 ------------------------------------- 1 file changed, 64 deletions(-) delete mode 100644 .github/workflows/docs.yaml diff --git a/.github/workflows/docs.yaml b/.github/workflows/docs.yaml deleted file mode 100644 index ab6a615ab60b..000000000000 --- a/.github/workflows/docs.yaml +++ /dev/null @@ -1,64 +0,0 @@ -on: - push: - branches: - - main - paths: - - .asf.yaml - - .github/workflows/docs.yaml - - docs/** - -name: Deploy DataFusion site - -jobs: - build-docs: - name: Build docs - runs-on: ubuntu-latest - steps: - - name: Checkout docs sources - uses: actions/checkout@v4 - - - name: Checkout asf-site branch - uses: actions/checkout@v4 - with: - ref: asf-site - path: asf-site - - - name: Setup Python - uses: actions/setup-python@v5 - with: - python-version: "3.10" - - - name: Install dependencies - run: | - set -x - python3 -m venv venv - source venv/bin/activate - pip install -r docs/requirements.txt - - - name: Build docs - run: | - set -x - source venv/bin/activate - cd docs - ./build.sh - - - name: Copy & push the generated HTML - run: | - set -x - cd asf-site/ - rsync \ - -a \ - --delete \ - --exclude '/.git/' \ - ../docs/build/html/ \ - ./ - cp ../.asf.yaml . - touch .nojekyll - git status --porcelain - if [ "$(git status --porcelain)" != "" ]; then - git config user.name "github-actions[bot]" - git config user.email "github-actions[bot]@users.noreply.github.com" - git add --all - git commit -m 'Publish built docs triggered by ${{ github.sha }}' - git push || git push --force - fi From ace9815ae5d8c99001c84fb71100536989364c34 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 22 Feb 2024 15:54:39 +0300 Subject: [PATCH 06/15] use create_cache_convention --- .../examples/custom_datasource.rs | 18 +-- .../datasource/physical_plan/arrow_file.rs | 34 ++--- .../core/src/datasource/physical_plan/avro.rs | 25 ++-- .../core/src/datasource/physical_plan/csv.rs | 35 ++--- .../core/src/datasource/physical_plan/json.rs | 35 ++--- .../datasource/physical_plan/parquet/mod.rs | 35 ++--- .../enforce_distribution.rs | 16 +-- .../physical_optimizer/output_requirements.rs | 16 +-- datafusion/core/src/physical_planner.rs | 19 +-- datafusion/core/src/test/mod.rs | 19 ++- datafusion/core/src/test_util/mod.rs | 27 ++-- datafusion/core/tests/custom_sources.rs | 20 +-- .../provider_filter_pushdown.rs | 21 ++- .../tests/custom_sources_cases/statistics.rs | 19 ++- .../tests/user_defined/user_defined_plan.rs | 20 +-- .../physical-plan/src/aggregates/mod.rs | 65 ++++----- datafusion/physical-plan/src/analyze.rs | 21 ++- .../physical-plan/src/coalesce_batches.rs | 17 +-- .../physical-plan/src/coalesce_partitions.rs | 15 +-- datafusion/physical-plan/src/empty.rs | 27 ++-- datafusion/physical-plan/src/explain.rs | 20 ++- datafusion/physical-plan/src/filter.rs | 112 ++++++++-------- datafusion/physical-plan/src/insert.rs | 26 ++-- .../physical-plan/src/joins/cross_join.rs | 36 ++--- .../physical-plan/src/joins/hash_join.rs | 42 +++--- .../src/joins/nested_loop_join.rs | 42 +++--- .../src/joins/sort_merge_join.rs | 42 +++--- .../src/joins/symmetric_hash_join.rs | 42 +++--- datafusion/physical-plan/src/lib.rs | 16 --- datafusion/physical-plan/src/limit.rs | 34 ++--- datafusion/physical-plan/src/memory.rs | 35 ++--- .../physical-plan/src/placeholder_row.rs | 29 ++-- datafusion/physical-plan/src/projection.rs | 31 ++--- .../physical-plan/src/recursive_query.rs | 21 +-- .../physical-plan/src/repartition/mod.rs | 60 ++++++--- .../physical-plan/src/sorts/partial_sort.rs | 48 ++++--- datafusion/physical-plan/src/sorts/sort.rs | 51 ++++--- .../src/sorts/sort_preserving_merge.rs | 17 +-- datafusion/physical-plan/src/streaming.rs | 34 +++-- datafusion/physical-plan/src/test/exec.rs | 124 ++++++++---------- datafusion/physical-plan/src/union.rs | 43 +++--- datafusion/physical-plan/src/unnest.rs | 24 ++-- datafusion/physical-plan/src/values.rs | 19 +-- .../src/windows/bounded_window_agg_exec.rs | 29 ++-- .../src/windows/window_agg_exec.rs | 23 ++-- datafusion/physical-plan/src/work_table.rs | 18 +-- 46 files changed, 787 insertions(+), 735 deletions(-) diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 9516dc570d6d..d3cd66b2c9bc 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -35,6 +35,7 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::*; use datafusion_expr::{Expr, LogicalPlanBuilder}; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use tokio::time::timeout; @@ -199,22 +200,21 @@ impl CustomExec { db: CustomDataSource, ) -> Self { let projected_schema = project_schema(&schema, projections).unwrap(); - let cache = PlanPropertiesCache::new_default(projected_schema.clone()); + let cache = Self::create_cache(projected_schema.clone()); Self { db, projected_schema, cache, } - .with_cache() } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - .with_partitioning(Partitioning::UnknownPartitioning(1)) - .with_exec_mode(ExecutionMode::Bounded); - - self + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 1a27f9315b34..24e825a6920b 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -61,7 +61,11 @@ impl ArrowExec { pub fn new(base_config: FileScanConfig) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = PlanPropertiesCache::new_default(projected_schema.clone()); + let cache = Self::create_cache( + projected_schema.clone(), + &projected_output_ordering, + &base_config, + ); Self { base_config, projected_schema, @@ -70,36 +74,36 @@ impl ArrowExec { metrics: ExecutionPlanMetricsSet::new(), cache, } - .with_cache() } /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config } - fn output_partitioning_helper(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) } - fn with_cache(mut self) -> Self { + fn create_cache( + schema: SchemaRef, + projected_output_ordering: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanPropertiesCache { // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ); + let eq_properties = + EquivalenceProperties::new_with_orderings(schema, projected_output_ordering); - self.cache = PlanPropertiesCache::new( + PlanPropertiesCache::new( eq_properties, - self.output_partitioning_helper(), // Output Partitioning - ExecutionMode::Bounded, // Execution Mode - ); - self + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) } 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(); + let output_partitioning = Self::output_partitioning_helper(&self.base_config); self.cache = self.cache.with_partitioning(output_partitioning); self } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index fb2cd627a1da..6b6e7bce90c1 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -50,7 +50,11 @@ impl AvroExec { pub fn new(base_config: FileScanConfig) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = PlanPropertiesCache::new_default(projected_schema.clone()); + let cache = Self::create_cache( + projected_schema.clone(), + &projected_output_ordering, + &base_config, + ); Self { base_config, projected_schema, @@ -59,27 +63,26 @@ impl AvroExec { metrics: ExecutionPlanMetricsSet::new(), cache, } - .with_cache() } /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config } - fn with_cache(mut self) -> Self { + fn create_cache( + schema: SchemaRef, + orderings: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanPropertiesCache { // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ); - let n_partitions = self.base_config.file_groups.len(); + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + let n_partitions = file_scan_config.file_groups.len(); - self.cache = PlanPropertiesCache::new( + PlanPropertiesCache::new( eq_properties, Partitioning::UnknownPartitioning(n_partitions), // Output Partitioning ExecutionMode::Bounded, // Execution Mode - ); - self + ) } } diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 55cf62507788..19281bc3c189 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -53,7 +53,6 @@ use tokio::task::JoinSet; pub struct CsvExec { base_config: FileScanConfig, projected_statistics: Statistics, - projected_output_ordering: Vec, has_header: bool, delimiter: u8, quote: u8, @@ -77,11 +76,14 @@ impl CsvExec { ) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = PlanPropertiesCache::new_default(projected_schema); + let cache = Self::create_cache( + projected_schema, + &projected_output_ordering, + &base_config, + ); Self { base_config, projected_statistics, - projected_output_ordering, has_header, delimiter, quote, @@ -90,7 +92,6 @@ impl CsvExec { file_compression_type, cache, } - .with_cache() } /// Ref to the base configs @@ -116,29 +117,29 @@ impl CsvExec { self.escape } - fn output_partitioning_helper(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) } - fn with_cache(mut self) -> Self { + fn create_cache( + schema: SchemaRef, + orderings: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanPropertiesCache { // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ); + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - self.cache = PlanPropertiesCache::new( + PlanPropertiesCache::new( eq_properties, - self.output_partitioning_helper(), // Output Partitioning - ExecutionMode::Bounded, // Execution Mode - ); - self + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) } 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(); + let output_partitioning = Self::output_partitioning_helper(&self.base_config); self.cache = self.cache.with_partitioning(output_partitioning); self } diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 231b48f28d96..6e17e58d8444 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -52,7 +52,6 @@ use tokio::task::JoinSet; pub struct NdJsonExec { base_config: FileScanConfig, projected_statistics: Statistics, - projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, file_compression_type: FileCompressionType, @@ -67,16 +66,18 @@ impl NdJsonExec { ) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = PlanPropertiesCache::new_default(projected_schema); + let cache = Self::create_cache( + projected_schema, + &projected_output_ordering, + &base_config, + ); Self { base_config, projected_statistics, - projected_output_ordering, metrics: ExecutionPlanMetricsSet::new(), file_compression_type, cache, } - .with_cache() } /// Ref to the base configs @@ -84,29 +85,29 @@ impl NdJsonExec { &self.base_config } - fn output_partitioning_helper(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) } - fn with_cache(mut self) -> Self { + fn create_cache( + schema: SchemaRef, + orderings: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanPropertiesCache { // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ); + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - self.cache = PlanPropertiesCache::new( + PlanPropertiesCache::new( eq_properties, - self.output_partitioning_helper(), // Output Partitioning - ExecutionMode::Bounded, // Execution Mode - ); - self + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) } 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(); + let output_partitioning = Self::output_partitioning_helper(&self.base_config); self.cache = self.cache.with_partitioning(output_partitioning); self } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index e47c8e516385..810a84646c86 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -88,7 +88,6 @@ pub struct ParquetExec { /// Base configuration for this scan base_config: FileScanConfig, projected_statistics: Statistics, - projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Optional predicate for row filtering during parquet scan @@ -149,7 +148,11 @@ impl ParquetExec { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = PlanPropertiesCache::new_default(projected_schema); + let cache = Self::create_cache( + projected_schema, + &projected_output_ordering, + &base_config, + ); Self { pushdown_filters: None, reorder_filters: None, @@ -157,7 +160,6 @@ impl ParquetExec { enable_bloom_filter: None, base_config, projected_statistics, - projected_output_ordering, metrics, predicate, pruning_predicate, @@ -166,7 +168,6 @@ impl ParquetExec { parquet_file_reader_factory: None, cache, } - .with_cache() } /// Ref to the base configs @@ -261,29 +262,29 @@ impl ParquetExec { .unwrap_or(config_options.execution.parquet.bloom_filter_enabled) } - fn output_partitioning_helper(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.base_config.file_groups.len()) + fn output_partitioning_helper(file_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_config.file_groups.len()) } - fn with_cache(mut self) -> Self { + fn create_cache( + schema: SchemaRef, + orderings: &[LexOrdering], + file_config: &FileScanConfig, + ) -> PlanPropertiesCache { // Equivalence Properties - let eq_properties = EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ); + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - self.cache = PlanPropertiesCache::new( + PlanPropertiesCache::new( eq_properties, - self.output_partitioning_helper(), // Output Partitioning - ExecutionMode::Bounded, // Execution Mode - ); - self + Self::output_partitioning_helper(file_config), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) } 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(); + let output_partitioning = Self::output_partitioning_helper(&self.base_config); self.cache = self.cache.with_partitioning(output_partitioning); self } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index c7bfe4742bdf..a5ad2d546d41 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1352,22 +1352,20 @@ pub(crate) mod tests { input: Arc, requirement: Vec, ) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); + let cache = Self::create_cache(&input); Self { input, expr: requirement, cache, } - .with_cache() } - fn with_cache(mut self) -> Self { - self.cache = PlanPropertiesCache::new( - self.input.equivalence_properties().clone(), // Equivalence Properties - self.input.output_partitioning().clone(), // Output Partitioning - self.input.execution_mode(), // Execution Mode - ); - self + fn create_cache(input: &Arc) -> PlanPropertiesCache { + PlanPropertiesCache::new( + input.equivalence_properties().clone(), // Equivalence Properties + input.output_partitioning().clone(), // Output Partitioning + input.execution_mode(), // Execution Mode + ) } } diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 5ddba79515ee..a806580ce716 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -99,27 +99,25 @@ impl OutputRequirementExec { requirements: Option, dist_requirement: Distribution, ) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); + let cache = Self::create_cache(&input); Self { input, order_requirement: requirements, dist_requirement, cache, } - .with_cache() } pub(crate) fn input(&self) -> Arc { self.input.clone() } - fn with_cache(mut self) -> Self { - self.cache = PlanPropertiesCache::new( - self.input.equivalence_properties().clone(), // Equivalence Properties - self.input.output_partitioning().clone(), // Output Partitioning - self.input.execution_mode(), // Execution Mode - ); - self + fn create_cache(input: &Arc) -> PlanPropertiesCache { + PlanPropertiesCache::new( + input.equivalence_properties().clone(), // Equivalence Properties + input.output_partitioning().clone(), // Output Partitioning + input.execution_mode(), // Execution Mode + ) } } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 1a334678b6d7..e571bc76f4d5 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2013,6 +2013,7 @@ mod tests { col, lit, sum, Extension, GroupingSet, LogicalPlanBuilder, UserDefinedLogicalNodeCore, }; + use datafusion_physical_expr::EquivalenceProperties; fn make_session_state() -> SessionState { let runtime = Arc::new(RuntimeEnv::default()); @@ -2579,19 +2580,19 @@ mod tests { impl NoOpExecutionPlan { fn new(schema: SchemaRef) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); - Self { cache }.with_cache() + let cache = Self::create_cache(schema.clone()); + Self { cache } } - fn with_cache(mut self) -> Self { - self.cache = self - .cache + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) + Partitioning::UnknownPartitioning(1), // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 277901ff9915..f8eb67cfdaf5 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -42,7 +42,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::{DataFusionError, FileType, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalSortExpr}; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionMode, PlanPropertiesCache, @@ -376,24 +376,23 @@ impl StatisticsExec { stats.column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); - let cache = PlanPropertiesCache::new_default(Arc::new(schema.clone())); + let cache = Self::create_cache(Arc::new(schema.clone())); Self { stats, schema: Arc::new(schema), cache, } - .with_cache() } - fn with_cache(mut self) -> Self { - self.cache = self - .cache + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(2)) + Partitioning::UnknownPartitioning(2), // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index bb016f93c351..dda6d730ce84 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -55,6 +55,7 @@ use tempfile::TempDir; #[cfg(feature = "parquet")] pub use datafusion_common::test_util::parquet_test_data; pub use datafusion_common::test_util::{arrow_test_data, get_data_dir}; +use datafusion_physical_expr::EquivalenceProperties; /// Scan an empty data source, mainly used in tests pub fn scan_empty( @@ -226,7 +227,6 @@ impl TableProvider for TestTableProvider { pub struct UnboundedExec { batch_produce: Option, batch: RecordBatch, - partitions: usize, cache: PlanPropertiesCache, } impl UnboundedExec { @@ -238,29 +238,30 @@ impl UnboundedExec { batch: RecordBatch, partitions: usize, ) -> Self { - let cache = PlanPropertiesCache::new_default(batch.schema()); + let cache = Self::create_cache(batch.schema(), batch_produce, partitions); Self { batch_produce, batch, - partitions, cache, } - .with_cache() } - fn with_cache(mut self) -> Self { - let mode = if self.batch_produce.is_none() { + fn create_cache( + schema: SchemaRef, + batch_produce: Option, + n_partitions: usize, + ) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + let mode = if batch_produce.is_none() { ExecutionMode::Unbounded } else { ExecutionMode::Bounded }; - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(self.partitions)) - // Execution Mode - .with_exec_mode(mode); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(n_partitions), + mode, + ) } } diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index a167258ee1d5..11f29192276c 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -38,6 +38,7 @@ use datafusion::scalar::ScalarValue; use datafusion_common::cast::as_primitive_array; use datafusion_common::project_schema; use datafusion_common::stats::Precision; +use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; @@ -81,19 +82,18 @@ impl CustomExecutionPlan { let schema = TEST_CUSTOM_SCHEMA_REF!(); let schema = project_schema(&schema, projection.as_ref()).expect("projected schema"); - let cache = PlanPropertiesCache::new_default(schema); - Self { projection, cache }.with_cache() + let cache = Self::create_cache(schema); + Self { projection, cache } } - fn with_cache(mut self) -> Self { - self.cache = self - .cache + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index 9423f0170c7e..da00effa00a8 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -35,6 +35,7 @@ use datafusion::scalar::ScalarValue; use datafusion_common::cast::as_primitive_array; use datafusion_common::{internal_err, not_impl_err, DataFusionError}; use datafusion_expr::expr::{BinaryExpr, Cast}; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; @@ -62,19 +63,17 @@ struct CustomPlan { impl CustomPlan { fn new(schema: SchemaRef, batches: Vec) -> Self { - let cache = PlanPropertiesCache::new_default(schema); - Self { batches, cache }.with_cache() + let cache = Self::create_cache(schema); + Self { batches, cache } } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 315c7cb6dd26..37854908f021 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -33,6 +33,7 @@ use datafusion::{ scalar::ScalarValue, }; use datafusion_common::{project_schema, stats::Precision}; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; @@ -52,24 +53,22 @@ impl StatisticsValidation { schema.fields().len(), "the column statistics vector length should be the number of fields" ); - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(schema.clone()); Self { stats, schema, cache, } - .with_cache() } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(2)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(2), + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 947376dfb6a0..f2b6f6c93615 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -91,6 +91,7 @@ use datafusion::{ }; use async_trait::async_trait; +use datafusion_physical_expr::EquivalenceProperties; use futures::{Stream, StreamExt}; /// Execute the specified sql and return the resulting record batches @@ -416,19 +417,18 @@ struct TopKExec { impl TopKExec { fn new(input: Arc, k: usize) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); - Self { input, k, cache }.with_cache() + let cache = Self::create_cache(input.schema()); + Self { input, k, cache } } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index ea3fc3a737b2..fa5b65e40123 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -260,9 +260,6 @@ pub struct AggregateExec { /// We need the input schema of partial aggregate to be able to deserialize aggregate /// expressions from protobuf for final aggregate. pub input_schema: SchemaRef, - /// The mapping used to normalize expressions like Partitioning and - /// PhysicalSortExpr that maps input to output - projection_mapping: ProjectionMapping, /// Execution metrics metrics: ExecutionPlanMetricsSet, required_input_ordering: Option, @@ -365,8 +362,14 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - let cache = PlanPropertiesCache::new_default(schema.clone()); - let aggregate = AggregateExec { + let cache = Self::create_cache( + &input, + schema.clone(), + &projection_mapping, + &mode, + &input_order_mode, + ); + Ok(AggregateExec { mode, group_by, aggr_expr, @@ -374,14 +377,12 @@ impl AggregateExec { input, schema, input_schema, - projection_mapping, metrics: ExecutionPlanMetricsSet::new(), required_input_ordering, limit: None, input_order_mode, cache, - }; - Ok(aggregate.with_cache()) + }) } /// Aggregation mode (full, partial) @@ -505,26 +506,31 @@ impl AggregateExec { true } - fn with_cache(mut self) -> Self { + fn create_cache( + input: &Arc, + schema: SchemaRef, + projection_mapping: &ProjectionMapping, + mode: &AggregateMode, + input_order_mode: &InputOrderMode, + ) -> PlanPropertiesCache { // Construct equivalence properties: - let eq_properties = self - .input + let eq_properties = input .equivalence_properties() - .project(&self.projection_mapping, self.schema()); + .project(projection_mapping, schema); // Get output partitioning: - let mut output_partitioning = self.input.output_partitioning().clone(); - if self.mode.is_first_stage() { + let mut output_partitioning = input.output_partitioning().clone(); + if mode.is_first_stage() { // First stage aggregation will not change the output partitioning, // but needs to respect aliases (e.g. mapping in the GROUP BY // expression). - let input_eq_properties = self.input.equivalence_properties(); + let input_eq_properties = input.equivalence_properties(); if let Partitioning::Hash(exprs, part) = output_partitioning { let normalized_exprs = exprs .iter() .map(|expr| { input_eq_properties - .project_expr(expr, &self.projection_mapping) + .project_expr(expr, projection_mapping) .unwrap_or_else(|| { Arc::new(UnKnownColumn::new(&expr.to_string())) }) @@ -535,18 +541,15 @@ impl AggregateExec { } // Determine execution mode: - let mut exec_mode = self.input.execution_mode(); + let mut exec_mode = input.execution_mode(); if exec_mode == ExecutionMode::Unbounded - && self.input_order_mode == InputOrderMode::Linear + && *input_order_mode == InputOrderMode::Linear { // Cannot run without breaking the pipeline exec_mode = ExecutionMode::PipelineBreaking; } - self.cache = - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode); - - self + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode) } pub fn input_order_mode(&self) -> &InputOrderMode { @@ -1622,19 +1625,19 @@ mod tests { impl TestYieldingExec { fn new(yield_first: bool) -> Self { let schema = some_data().0; - let cache = PlanPropertiesCache::new_default(schema); - Self { yield_first, cache }.with_cache() + let cache = Self::create_cache(schema); + Self { yield_first, cache } } - fn with_cache(mut self) -> Self { - self.cache = self - .cache + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) + Partitioning::UnknownPartitioning(1), // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 99b3add2acd6..731f3e3c7ebf 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -29,6 +29,7 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use futures::StreamExt; @@ -55,7 +56,7 @@ impl AnalyzeExec { input: Arc, schema: SchemaRef, ) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(&input, schema.clone()); AnalyzeExec { verbose, show_statistics, @@ -63,7 +64,6 @@ impl AnalyzeExec { schema, cache, } - .with_cache() } /// access to verbose @@ -81,15 +81,14 @@ impl AnalyzeExec { &self.input } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) - // Execution Mode - .with_exec_mode(self.input.execution_mode()); - - self + fn create_cache( + input: &Arc, + schema: SchemaRef, + ) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + let output_partitioning = Partitioning::UnknownPartitioning(1); + let exec_mode = input.execution_mode(); + PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode) } } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index e01060f3784d..e83bce0664a3 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -54,14 +54,13 @@ pub struct CoalesceBatchesExec { impl CoalesceBatchesExec { /// Create a new CoalesceBatchesExec pub fn new(input: Arc, target_batch_size: usize) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); + let cache = Self::create_cache(&input); Self { input, target_batch_size, metrics: ExecutionPlanMetricsSet::new(), cache, } - .with_cache() } /// The input plan @@ -74,16 +73,14 @@ impl CoalesceBatchesExec { self.target_batch_size } - fn with_cache(mut self) -> Self { + fn create_cache(input: &Arc) -> PlanPropertiesCache { // The coalesce batches operator does not make any changes to the // partitioning of its input. - self.cache = PlanPropertiesCache::new( - self.input.equivalence_properties().clone(), // Equivalence Properties - self.input.output_partitioning().clone(), // Output Partitioning - self.input.execution_mode(), // Execution Mode - ); - - self + PlanPropertiesCache::new( + input.equivalence_properties().clone(), // Equivalence Properties + input.output_partitioning().clone(), // Output Partitioning + input.execution_mode(), // Execution Mode + ) } } diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 255e996bd122..27f58c9bfd85 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -44,13 +44,12 @@ pub struct CoalescePartitionsExec { impl CoalescePartitionsExec { /// Create a new CoalescePartitionsExec pub fn new(input: Arc) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); + let cache = Self::create_cache(&input); CoalescePartitionsExec { input, metrics: ExecutionPlanMetricsSet::new(), cache, } - .with_cache() } /// Input execution plan @@ -58,18 +57,16 @@ impl CoalescePartitionsExec { &self.input } - fn with_cache(mut self) -> Self { + fn create_cache(input: &Arc) -> PlanPropertiesCache { // Coalescing partitions loses existing orderings: - let mut eq_properties = self.input.equivalence_properties().clone(); + let mut eq_properties = input.equivalence_properties().clone(); eq_properties.clear_orderings(); - self.cache = PlanPropertiesCache::new( + PlanPropertiesCache::new( eq_properties, // Equivalence Properties Partitioning::UnknownPartitioning(1), // Output Partitioning - self.input.execution_mode(), // Execution Mode - ); - - self + input.execution_mode(), // Execution Mode + ) } } diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index d91395825135..942bee81f472 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -30,6 +30,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use log::trace; @@ -46,20 +47,19 @@ pub struct EmptyExec { impl EmptyExec { /// Create a new EmptyExec pub fn new(schema: SchemaRef) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(schema.clone(), 1); EmptyExec { schema, partitions: 1, cache, } - .with_cache() } /// Create a new EmptyExec with specified partition number pub fn with_partitions(mut self, partitions: usize) -> Self { self.partitions = partitions; // Changing partitions may invalidate output partitioning, so update it: - let output_partitioning = self.output_partitioning_helper(); + let output_partitioning = Self::output_partitioning_helper(self.partitions); self.cache = self.cache.with_partitioning(output_partitioning); self } @@ -68,21 +68,20 @@ impl EmptyExec { Ok(vec![]) } - fn output_partitioning_helper(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.partitions) + fn output_partitioning_helper(n_partitions: usize) -> Partitioning { + Partitioning::UnknownPartitioning(n_partitions) } - fn with_cache(mut self) -> Self { - let output_partitioning = self.output_partitioning_helper(); - - self.cache = self - .cache + fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + let output_partitioning = Self::output_partitioning_helper(n_partitions); + PlanPropertiesCache::new( + eq_properties, // Output Partitioning - .with_partitioning(output_partitioning) + output_partitioning, // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 935b37c168da..689ef32aa1a9 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -28,6 +28,7 @@ use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatc use datafusion_common::display::StringifiedPlan; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use log::trace; @@ -52,14 +53,13 @@ impl ExplainExec { stringified_plans: Vec, verbose: bool, ) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(schema.clone()); ExplainExec { schema, stringified_plans, verbose, cache, } - .with_cache() } /// The strings to be printed @@ -72,15 +72,13 @@ impl ExplainExec { self.verbose } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index cc8fdcbcd0cd..d6942f0d5678 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -73,15 +73,15 @@ impl FilterExec { ) -> Result { match predicate.data_type(input.schema().as_ref())? { DataType::Boolean => { - let cache = PlanPropertiesCache::new_default(input.schema()); + let default_selectivity = 20; + let cache = Self::create_cache(&input, &predicate, default_selectivity)?; Ok(Self { predicate, input: input.clone(), metrics: ExecutionPlanMetricsSet::new(), - default_selectivity: 20, + default_selectivity, cache, - } - .with_cache()) + }) } other => { plan_err!("Filter predicate must return boolean values, not {other:?}") @@ -115,12 +115,58 @@ impl FilterExec { self.default_selectivity } - fn with_cache(mut self) -> Self { + fn statistics_helper( + input: &Arc, + predicate: &Arc, + default_selectivity: u8, + ) -> Result { + let input_stats = input.statistics()?; + let schema = input.schema(); + if !check_support(predicate, &schema) { + let selectivity = default_selectivity as f64 / 100.0; + let mut stats = input_stats.into_inexact(); + stats.num_rows = stats.num_rows.with_estimated_selectivity(selectivity); + stats.total_byte_size = stats + .total_byte_size + .with_estimated_selectivity(selectivity); + return Ok(stats); + } + + let num_rows = input_stats.num_rows; + let total_byte_size = input_stats.total_byte_size; + let input_analysis_ctx = AnalysisContext::try_from_statistics( + &input.schema(), + &input_stats.column_statistics, + )?; + + let analysis_ctx = analyze(predicate, input_analysis_ctx, &schema)?; + + // Estimate (inexact) selectivity of predicate + let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); + let num_rows = num_rows.with_estimated_selectivity(selectivity); + let total_byte_size = total_byte_size.with_estimated_selectivity(selectivity); + + let column_statistics = collect_new_statistics( + &input_stats.column_statistics, + analysis_ctx.boundaries, + ); + Ok(Statistics { + num_rows, + total_byte_size, + column_statistics, + }) + } + + fn create_cache( + input: &Arc, + predicate: &Arc, + default_selectivity: u8, + ) -> Result { // Combine the equal predicates with the input equivalence properties // to construct the equivalence properties: - let stats = self.statistics().unwrap(); - let mut eq_properties = self.input.equivalence_properties().clone(); - let (equal_pairs, _) = collect_columns_from_predicate(&self.predicate); + let stats = Self::statistics_helper(input, predicate, default_selectivity)?; + let mut eq_properties = input.equivalence_properties().clone(); + let (equal_pairs, _) = collect_columns_from_predicate(predicate); for (lhs, rhs) in equal_pairs { let lhs_expr = Arc::new(lhs.clone()) as _; let rhs_expr = Arc::new(rhs.clone()) as _; @@ -128,19 +174,17 @@ impl FilterExec { } // Add the columns that have only one viable value (singleton) after // filtering to constants. - let constants = collect_columns(self.predicate()) + let constants = collect_columns(predicate) .into_iter() .filter(|column| stats.column_statistics[column.index()].is_singleton()) .map(|column| Arc::new(column) as _); eq_properties = eq_properties.add_constants(constants); - self.cache = PlanPropertiesCache::new( + Ok(PlanPropertiesCache::new( eq_properties, - self.input.output_partitioning().clone(), // Output Partitioning - self.input.execution_mode(), // Execution Mode - ); - - self + input.output_partitioning().clone(), // Output Partitioning + input.execution_mode(), // Execution Mode + )) } } @@ -211,43 +255,7 @@ impl ExecutionPlan for FilterExec { /// The output statistics of a filtering operation can be estimated if the /// predicate's selectivity value can be determined for the incoming data. fn statistics(&self) -> Result { - let predicate = self.predicate(); - - let input_stats = self.input.statistics()?; - let schema = self.schema(); - if !check_support(predicate, &schema) { - let selectivity = self.default_selectivity as f64 / 100.0; - let mut stats = input_stats.into_inexact(); - stats.num_rows = stats.num_rows.with_estimated_selectivity(selectivity); - stats.total_byte_size = stats - .total_byte_size - .with_estimated_selectivity(selectivity); - return Ok(stats); - } - - let num_rows = input_stats.num_rows; - let total_byte_size = input_stats.total_byte_size; - let input_analysis_ctx = AnalysisContext::try_from_statistics( - &self.input.schema(), - &input_stats.column_statistics, - )?; - - let analysis_ctx = analyze(predicate, input_analysis_ctx, &self.schema())?; - - // Estimate (inexact) selectivity of predicate - let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); - let num_rows = num_rows.with_estimated_selectivity(selectivity); - let total_byte_size = total_byte_size.with_estimated_selectivity(selectivity); - - let column_statistics = collect_new_statistics( - &input_stats.column_statistics, - analysis_ctx.boundaries, - ); - Ok(Statistics { - num_rows, - total_byte_size, - column_statistics, - }) + Self::statistics_helper(&self.input, self.predicate(), self.default_selectivity) } } diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index b20e8cac7926..472c65f25b30 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -35,7 +35,9 @@ use arrow_array::{ArrayRef, UInt64Array}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{exec_err, internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{Distribution, PhysicalSortRequirement}; +use datafusion_physical_expr::{ + Distribution, EquivalenceProperties, PhysicalSortRequirement, +}; use async_trait::async_trait; use futures::StreamExt; @@ -104,7 +106,7 @@ impl FileSinkExec { sort_order: Option>, ) -> Self { let count_schema = make_count_schema(); - let cache = PlanPropertiesCache::new_default(count_schema); + let cache = Self::create_schema(&input, count_schema); Self { input, sink, @@ -113,7 +115,6 @@ impl FileSinkExec { sort_order, cache, } - .with_cache() } fn execute_input_stream( @@ -176,15 +177,16 @@ impl FileSinkExec { self.sink.metrics() } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) - // Execution Mode - .with_exec_mode(self.input.execution_mode()); - - self + fn create_schema( + input: &Arc, + schema: SchemaRef, + ) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + input.execution_mode(), + ) } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 8b12b02b4667..42758e635060 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -27,9 +27,9 @@ use crate::coalesce_batches::concat_batches; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, - ExecutionPlan, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, - Statistics, + exec_mode_flatten, ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, + ExecutionMode, ExecutionPlan, PlanPropertiesCache, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; use arrow::datatypes::{Fields, Schema, SchemaRef}; @@ -77,7 +77,7 @@ impl CrossJoinExec { }; let schema = Arc::new(Schema::new(all_columns)); - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(&left, &right, schema.clone()); CrossJoinExec { left, right, @@ -86,7 +86,6 @@ impl CrossJoinExec { metrics: ExecutionPlanMetricsSet::default(), cache, } - .with_cache() } /// left (build) side which gets loaded in memory @@ -99,15 +98,19 @@ impl CrossJoinExec { &self.right } - fn with_cache(mut self) -> Self { + fn create_cache( + left: &Arc, + right: &Arc, + schema: SchemaRef, + ) -> PlanPropertiesCache { // Calculate equivalence properties // TODO: Check equivalence properties of cross join, it may preserve // ordering in some cases. let eq_properties = join_equivalence_properties( - self.left.equivalence_properties().clone(), - self.right.equivalence_properties().clone(), + left.equivalence_properties().clone(), + right.equivalence_properties().clone(), &JoinType::Full, - self.schema(), + schema, &[false, false], None, &[], @@ -117,19 +120,18 @@ impl CrossJoinExec { // TODO: Optimize the cross join implementation to generate M * N // partitions. let output_partitioning = adjust_right_output_partitioning( - self.right.output_partitioning(), - self.left.schema().fields.len(), + right.output_partitioning(), + left.schema().fields.len(), ); // Determine the execution mode: - let mode = match (self.left.execution_mode(), self.right.execution_mode()) { - (ExecutionMode::Bounded, ExecutionMode::Bounded) => ExecutionMode::Bounded, + let mut mode = exec_mode_flatten([left, right]); + if mode.is_unbounded() { // If any of the inputs is unbounded, cross join breaks the pipeline. - (_, _) => ExecutionMode::PipelineBreaking, - }; + mode = ExecutionMode::PipelineBreaking; + } - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 3aae053151cd..2b88ec449a04 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -65,6 +65,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::PhysicalExprRef; +use crate::joins::utils::JoinOnRef; use ahash::RandomState; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -327,7 +328,14 @@ impl HashJoinExec { let random_state = RandomState::with_seeds(0, 0, 0, 0); - let cache = PlanPropertiesCache::new_default(Arc::new(schema.clone())); + let cache = Self::create_cache( + &left, + &right, + Arc::new(schema.clone()), + *join_type, + &on, + partition_mode, + ); Ok(HashJoinExec { left, @@ -343,8 +351,7 @@ impl HashJoinExec { column_indices, null_equals_null, cache, - } - .with_cache()) + }) } /// left (build) side which gets hashed @@ -399,25 +406,29 @@ impl HashJoinExec { JoinSide::Right } - fn with_cache(mut self) -> Self { - let left = &self.left; - let right = &self.right; - let schema = self.schema(); + fn create_cache( + left: &Arc, + right: &Arc, + schema: SchemaRef, + join_type: JoinType, + on: JoinOnRef, + mode: PartitionMode, + ) -> PlanPropertiesCache { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( left.equivalence_properties().clone(), right.equivalence_properties().clone(), - &self.join_type, + &join_type, schema, - &Self::maintains_input_order(self.join_type), + &Self::maintains_input_order(join_type), Some(Self::probe_side()), - &self.on, + on, ); // Get output partitioning: let left_columns_len = left.schema().fields.len(); - let output_partitioning = match self.mode { - PartitionMode::CollectLeft => match self.join_type { + let output_partitioning = match mode { + PartitionMode::CollectLeft => match join_type { JoinType::Inner | JoinType::Right => adjust_right_output_partitioning( right.output_partitioning(), left_columns_len, @@ -433,7 +444,7 @@ impl HashJoinExec { ), }, PartitionMode::Partitioned => partitioned_join_output_partitioning( - self.join_type, + join_type, left.output_partitioning(), right.output_partitioning(), left_columns_len, @@ -449,7 +460,7 @@ impl HashJoinExec { let pipeline_breaking = left.execution_mode().is_unbounded() || (right.execution_mode().is_unbounded() && matches!( - self.join_type, + join_type, JoinType::Left | JoinType::Full | JoinType::LeftAnti @@ -462,8 +473,7 @@ impl HashJoinExec { exec_mode_flatten([left, right]) }; - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 1978338d2b6a..89beac14816d 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -109,19 +109,19 @@ impl NestedLoopJoinExec { check_join_is_valid(&left_schema, &right_schema, &[])?; let (schema, column_indices) = build_join_schema(&left_schema, &right_schema, join_type); - let cache = PlanPropertiesCache::new_default(Arc::new(schema.clone())); + let schema = Arc::new(schema); + let cache = Self::create_cache(&left, &right, schema.clone(), *join_type); Ok(NestedLoopJoinExec { left, right, filter, join_type: *join_type, - schema: Arc::new(schema), + schema, inner_table: Default::default(), column_indices, metrics: Default::default(), cache, - } - .with_cache()) + }) } /// left side @@ -144,39 +144,43 @@ impl NestedLoopJoinExec { &self.join_type } - fn with_cache(mut self) -> Self { + fn create_cache( + left: &Arc, + right: &Arc, + schema: SchemaRef, + join_type: JoinType, + ) -> PlanPropertiesCache { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( - self.left.equivalence_properties().clone(), - self.right.equivalence_properties().clone(), - &self.join_type, - self.schema(), - &self.maintains_input_order(), + left.equivalence_properties().clone(), + right.equivalence_properties().clone(), + &join_type, + schema, + &[false, false], None, // No on columns in nested loop join &[], ); // Get output partitioning, - let output_partitioning = if self.join_type == JoinType::Full { - self.left.output_partitioning().clone() + let output_partitioning = if join_type == JoinType::Full { + left.output_partitioning().clone() } else { partitioned_join_output_partitioning( - self.join_type, - self.left.output_partitioning(), - self.right.output_partitioning(), - self.left.schema().fields.len(), + join_type, + left.output_partitioning(), + right.output_partitioning(), + left.schema().fields.len(), ) }; // Determine execution mode: - let mut mode = exec_mode_flatten([&self.left, &self.right]); + let mut mode = exec_mode_flatten([left, right]); if mode.is_unbounded() { mode = ExecutionMode::PipelineBreaking; } - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 0d7cd995a5a2..f7d754a99e0e 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -33,7 +33,7 @@ use std::task::{Context, Poll}; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ build_join_schema, check_join_is_valid, estimate_join_statistics, - partitioned_join_output_partitioning, JoinFilter, JoinOn, + partitioned_join_output_partitioning, JoinFilter, JoinOn, JoinOnRef, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ @@ -137,7 +137,7 @@ impl SortMergeJoinExec { let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(&left, &right, schema.clone(), join_type, &on); Ok(Self { left, right, @@ -151,8 +151,7 @@ impl SortMergeJoinExec { sort_options, null_equals_null, cache, - } - .with_cache()) + }) } /// Get probe side (e.g streaming side) information for this sort merge join. @@ -201,32 +200,37 @@ impl SortMergeJoinExec { self.left.as_ref() } - fn with_cache(mut self) -> Self { + fn create_cache( + left: &Arc, + right: &Arc, + schema: SchemaRef, + join_type: JoinType, + join_on: JoinOnRef, + ) -> PlanPropertiesCache { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( - self.left.equivalence_properties().clone(), - self.right.equivalence_properties().clone(), - &self.join_type, - self.schema(), - &self.maintains_input_order(), - Some(Self::probe_side(&self.join_type)), - self.on(), + left.equivalence_properties().clone(), + right.equivalence_properties().clone(), + &join_type, + schema, + &Self::maintains_input_order(join_type), + Some(Self::probe_side(&join_type)), + join_on, ); // Get output partitioning: - let left_columns_len = self.left.schema().fields.len(); + let left_columns_len = left.schema().fields.len(); let output_partitioning = partitioned_join_output_partitioning( - self.join_type, - self.left.output_partitioning(), - self.right.output_partitioning(), + join_type, + left.output_partitioning(), + right.output_partitioning(), left_columns_len, ); // Determine execution mode: - let mode = exec_mode_flatten([&self.left, &self.right]); + let mode = exec_mode_flatten([left, right]); - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 5fd89e98a58f..4e07b10dd517 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -43,7 +43,7 @@ use crate::joins::stream_join_utils::{ use crate::joins::utils::{ apply_join_filter_to_indices, build_batch_from_indices, build_join_schema, check_join_is_valid, partitioned_join_output_partitioning, ColumnIndex, JoinFilter, - JoinHashMapType, JoinOn, StatefulStreamResult, + JoinHashMapType, JoinOn, JoinOnRef, StatefulStreamResult, }; use crate::{ exec_mode_flatten, @@ -233,8 +233,8 @@ impl SymmetricHashJoinExec { // Initialize the random state for the join operation: let random_state = RandomState::with_seeds(0, 0, 0, 0); - - let cache = PlanPropertiesCache::new_default(Arc::new(schema)); + let schema = Arc::new(schema); + let cache = Self::create_cache(&left, &right, schema.clone(), *join_type, &on); Ok(SymmetricHashJoinExec { left, right, @@ -249,37 +249,41 @@ impl SymmetricHashJoinExec { right_sort_exprs, mode, cache, - } - .with_cache()) + }) } - fn with_cache(mut self) -> Self { + fn create_cache( + left: &Arc, + right: &Arc, + schema: SchemaRef, + join_type: JoinType, + join_on: JoinOnRef, + ) -> PlanPropertiesCache { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( - self.left.equivalence_properties().clone(), - self.right.equivalence_properties().clone(), - &self.join_type, - self.schema(), - &self.maintains_input_order(), + left.equivalence_properties().clone(), + right.equivalence_properties().clone(), + &join_type, + schema, + &[false, false], // Has alternating probe side None, - self.on(), + join_on, ); // Get output partitioning: - let left_columns_len = self.left.schema().fields.len(); + let left_columns_len = left.schema().fields.len(); let output_partitioning = partitioned_join_output_partitioning( - self.join_type, - self.left.output_partitioning(), - self.right.output_partitioning(), + join_type, + left.output_partitioning(), + right.output_partitioning(), left_columns_len, ); // Determine execution mode: - let mode = exec_mode_flatten([&self.left, &self.right]); + let mode = exec_mode_flatten([left, right]); - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } /// left stream diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 52bfb0063d40..f90bbf061d38 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -540,22 +540,6 @@ impl PlanPropertiesCache { } } - /// Construct a default `PlanPropertiesCache`, for a given schema. - pub fn new_default(schema: SchemaRef) -> PlanPropertiesCache { - // Default values are the most restrictive possible values. - let eq_properties = EquivalenceProperties::new(schema); - // Please note that this default is not safe, and should be overwritten. - let partitioning = Partitioning::UnknownPartitioning(0); - let exec_mode = ExecutionMode::PipelineBreaking; - let output_ordering = None; - Self { - eq_properties, - partitioning, - exec_mode, - output_ordering, - } - } - /// Overwrite output partitioning with its new value. pub fn with_partitioning(mut self, partitioning: Partitioning) -> Self { self.partitioning = partitioning; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 59559e84fd75..a4b924d71066 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -57,7 +57,7 @@ pub struct GlobalLimitExec { impl GlobalLimitExec { /// Create a new GlobalLimitExec pub fn new(input: Arc, skip: usize, fetch: Option) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); + let cache = Self::create_cache(&input); GlobalLimitExec { input, skip, @@ -65,7 +65,6 @@ impl GlobalLimitExec { metrics: ExecutionPlanMetricsSet::new(), cache, } - .with_cache() } /// Input execution plan @@ -83,14 +82,12 @@ impl GlobalLimitExec { self.fetch } - fn with_cache(mut self) -> Self { - self.cache = PlanPropertiesCache::new( - self.input.equivalence_properties().clone(), // Equivalence Properties - Partitioning::UnknownPartitioning(1), // Output Partitioning - ExecutionMode::Bounded, // Execution Mode - ); - - self + fn create_cache(input: &Arc) -> PlanPropertiesCache { + PlanPropertiesCache::new( + input.equivalence_properties().clone(), // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) } } @@ -276,14 +273,13 @@ pub struct LocalLimitExec { impl LocalLimitExec { /// Create a new LocalLimitExec partition pub fn new(input: Arc, fetch: usize) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); + let cache = Self::create_cache(&input); Self { input, fetch, metrics: ExecutionPlanMetricsSet::new(), cache, } - .with_cache() } /// Input execution plan @@ -296,14 +292,12 @@ impl LocalLimitExec { self.fetch } - fn with_cache(mut self) -> Self { - self.cache = PlanPropertiesCache::new( - self.input.equivalence_properties().clone(), // Equivalence Properties - self.input.output_partitioning().clone(), // Output Partitioning - ExecutionMode::Bounded, // Execution Mode - ); - - self + fn create_cache(input: &Arc) -> PlanPropertiesCache { + PlanPropertiesCache::new( + input.equivalence_properties().clone(), // Equivalence Properties + input.output_partitioning().clone(), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) } } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 206d22e72226..f6039ee8b3ed 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -153,7 +153,7 @@ impl MemoryExec { projection: Option>, ) -> Result { let projected_schema = project_schema(&schema, projection.as_ref())?; - let cache = PlanPropertiesCache::new_default(projected_schema.clone()); + let cache = Self::create_cache(projected_schema.clone(), &[], partitions); Ok(Self { partitions: partitions.to_vec(), schema, @@ -161,8 +161,7 @@ impl MemoryExec { projection, sort_information: vec![], cache, - } - .with_cache()) + }) } pub fn partitions(&self) -> &[Vec] { @@ -193,27 +192,29 @@ impl MemoryExec { self.sort_information = sort_information; // We need to update equivalence properties when updating sort information. - let eq_properties = self.equivalent_properties_helper(); + let eq_properties = EquivalenceProperties::new_with_orderings( + self.schema(), + &self.sort_information, + ); self.cache = self.cache.with_eq_properties(eq_properties); - self.with_cache() + self } pub fn original_schema(&self) -> SchemaRef { self.schema.clone() } - fn equivalent_properties_helper(&self) -> EquivalenceProperties { - EquivalenceProperties::new_with_orderings(self.schema(), &self.sort_information) - } - - fn with_cache(mut self) -> Self { - self.cache = PlanPropertiesCache::new( - self.equivalent_properties_helper(), // Equivalence Properties - Partitioning::UnknownPartitioning(self.partitions.len()), // Output Partitioning - ExecutionMode::Bounded, // Execution Mode - ); - - self + fn create_cache( + schema: SchemaRef, + orderings: &[LexOrdering], + partitions: &[Vec], + ) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + PlanPropertiesCache::new( + eq_properties, // Equivalence Properties + Partitioning::UnknownPartitioning(partitions.len()), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) } } diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index a9b27cb13fe1..9d4204ddb589 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -32,6 +32,7 @@ use arrow::record_batch::RecordBatch; use arrow_array::RecordBatchOptions; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use log::trace; @@ -48,20 +49,20 @@ pub struct PlaceholderRowExec { impl PlaceholderRowExec { /// Create a new PlaceholderRowExec pub fn new(schema: SchemaRef) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let partitions = 1; + let cache = Self::create_cache(schema.clone(), partitions); PlaceholderRowExec { schema, - partitions: 1, + partitions, cache, } - .with_cache() } /// Create a new PlaceholderRowExecPlaceholderRowExec with specified partition number pub fn with_partitions(mut self, partitions: usize) -> Self { self.partitions = partitions; // Update output partitioning when updating partitions: - let output_partitioning = self.output_partitioning_helper(); + let output_partitioning = Self::output_partitioning_helper(self.partitions); self.cache = self.cache.with_partitioning(output_partitioning); self } @@ -89,20 +90,20 @@ impl PlaceholderRowExec { }) } - fn output_partitioning_helper(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.partitions) + fn output_partitioning_helper(n_partitions: usize) -> Partitioning { + Partitioning::UnknownPartitioning(n_partitions) } - fn with_cache(mut self) -> Self { + fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); // Get output partitioning: - let output_partitioning = self.output_partitioning_helper(); + let output_partitioning = Self::output_partitioning_helper(n_partitions); - self.cache = self - .cache - .with_partitioning(output_partitioning) - .with_exec_mode(ExecutionMode::Bounded); - - self + PlanPropertiesCache::new( + eq_properties, + output_partitioning, + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 4c17aa3d834a..7420cf58b5ce 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -56,9 +56,6 @@ pub struct ProjectionExec { schema: SchemaRef, /// The input plan input: Arc, - /// The mapping used to normalize expressions like Partitioning and - /// PhysicalSortExpr that maps input to output - projection_mapping: ProjectionMapping, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. @@ -96,16 +93,14 @@ impl ProjectionExec { // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; - let cache = PlanPropertiesCache::new_default(schema.clone()); - let projection = Self { + let cache = Self::create_cache(&input, &projection_mapping, schema.clone())?; + Ok(Self { expr, schema, input, - projection_mapping, metrics: ExecutionPlanMetricsSet::new(), cache, - }; - projection.with_cache() + }) } /// The projection expressions stored as tuples of (expression, output column name) @@ -118,13 +113,15 @@ impl ProjectionExec { &self.input } - fn with_cache(mut self) -> Result { - let input = &self.input; + fn create_cache( + input: &Arc, + projection_mapping: &ProjectionMapping, + schema: SchemaRef, + ) -> Result { // Calculate equivalence properties: let mut input_eq_properties = input.equivalence_properties().clone(); - input_eq_properties.substitute_oeq_class(&self.projection_mapping)?; - let eq_properties = - input_eq_properties.project(&self.projection_mapping, self.schema.clone()); + input_eq_properties.substitute_oeq_class(projection_mapping)?; + let eq_properties = input_eq_properties.project(projection_mapping, schema); // Calculate output partitioning, which needs to respect aliases: let input_partition = input.output_partitioning(); @@ -134,7 +131,7 @@ impl ProjectionExec { .iter() .map(|expr| { input_eq_properties - .project_expr(expr, &self.projection_mapping) + .project_expr(expr, projection_mapping) .unwrap_or_else(|| { Arc::new(UnKnownColumn::new(&expr.to_string())) }) @@ -145,13 +142,11 @@ impl ProjectionExec { input_partition.clone() }; - self.cache = PlanPropertiesCache::new( + Ok(PlanPropertiesCache::new( eq_properties, output_partitioning, input.execution_mode(), - ); - - Ok(self) + )) } } diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index 97a626c25116..adc675ba2730 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -33,7 +33,8 @@ use arrow::record_batch::RecordBatch; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::Partitioning; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; + use futures::{ready, Stream, StreamExt}; /// Recursive query execution plan. @@ -81,7 +82,7 @@ impl RecursiveQueryExec { let work_table = Arc::new(WorkTable::new()); // Use the same work table for both the WorkTableExec and the recursive term let recursive_term = assign_work_table(recursive_term, work_table.clone())?; - let cache = PlanPropertiesCache::new_default(static_term.schema()); + let cache = Self::create_cache(static_term.schema()); Ok(RecursiveQueryExec { name, static_term, @@ -90,17 +91,17 @@ impl RecursiveQueryExec { work_table, metrics: ExecutionPlanMetricsSet::new(), cache, - } - .with_cache()) + }) } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - .with_partitioning(Partitioning::UnknownPartitioning(1)) - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 6d2835df05f0..dc1e88f52e56 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -44,7 +44,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; use futures::stream::Stream; use futures::{FutureExt, StreamExt}; @@ -436,12 +436,7 @@ impl ExecutionPlan for RepartitionExec { } fn maintains_input_order(&self) -> Vec { - if self.preserve_order { - vec![true] - } else { - // We preserve ordering when input partitioning is 1 - vec![self.input().output_partitioning().partition_count() <= 1] - } + Self::maintains_input_order_helper(self.input(), self.preserve_order) } fn execute( @@ -602,7 +597,8 @@ impl RepartitionExec { input: Arc, partitioning: Partitioning, ) -> Result { - let cache = PlanPropertiesCache::new_default(input.schema()); + let preserve_order = false; + let cache = Self::create_cache(&input, partitioning.clone(), preserve_order); Ok(RepartitionExec { input, partitioning, @@ -611,27 +607,49 @@ impl RepartitionExec { abort_helper: Arc::new(AbortOnDropMany::<()>(vec![])), })), metrics: ExecutionPlanMetricsSet::new(), - preserve_order: false, + preserve_order, cache, + }) + } + + fn maintains_input_order_helper( + input: &Arc, + preserve_order: bool, + ) -> Vec { + if preserve_order { + vec![true] + } else { + // We preserve ordering when input partitioning is 1 + vec![input.output_partitioning().partition_count() <= 1] } - .with_cache()) } - fn with_cache(mut self) -> Self { + fn eq_properties_helper( + input: &Arc, + preserve_order: bool, + ) -> EquivalenceProperties { // Equivalence Properties - let mut eq_properties = self.input.equivalence_properties().clone(); + let mut eq_properties = input.equivalence_properties().clone(); // If the ordering is lost, reset the ordering equivalence class: - if !self.maintains_input_order()[0] { + if !Self::maintains_input_order_helper(input, preserve_order)[0] { eq_properties.clear_orderings(); } + eq_properties + } - self.cache = PlanPropertiesCache::new( - eq_properties, // Equivalence Properties - self.partitioning.clone(), // Output Partitioning - self.input.execution_mode(), // Execution Mode - ); + fn create_cache( + input: &Arc, + partitioning: Partitioning, + preserve_order: bool, + ) -> PlanPropertiesCache { + // Equivalence Properties + let eq_properties = Self::eq_properties_helper(input, preserve_order); - self + PlanPropertiesCache::new( + eq_properties, // Equivalence Properties + partitioning, // Output Partitioning + input.execution_mode(), // Execution Mode + ) } /// Specify if this reparititoning operation should preserve the order of @@ -648,7 +666,9 @@ impl RepartitionExec { // if there is only one input partition, merging is not required // to maintain order self.input.output_partitioning().partition_count() > 1; - self.with_cache() + let eq_properties = Self::eq_properties_helper(&self.input, self.preserve_order); + self.cache = self.cache.with_eq_properties(eq_properties); + self } /// Return the sort expressions that are used to merge diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 99ead9f904a1..16c4bc8601b6 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -71,6 +71,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::Result; use datafusion_execution::{RecordBatchStream, TaskContext}; +use datafusion_physical_expr::LexOrdering; use futures::{ready, Stream, StreamExt}; use log::trace; @@ -104,17 +105,17 @@ impl PartialSortExec { common_prefix_length: usize, ) -> Self { assert!(common_prefix_length > 0); - let cache = PlanPropertiesCache::new_default(input.schema()); + let preserve_partitioning = false; + let cache = Self::create_cache(&input, expr.clone(), preserve_partitioning); Self { input, expr, common_prefix_length, metrics_set: ExecutionPlanMetricsSet::new(), - preserve_partitioning: false, + preserve_partitioning, fetch: None, cache, } - .with_cache() } /// Whether this `PartialSortExec` preserves partitioning of the children @@ -131,6 +132,12 @@ impl PartialSortExec { /// input partitions producing a single, sorted partition. pub fn with_preserve_partitioning(mut self, preserve_partitioning: bool) -> Self { self.preserve_partitioning = preserve_partitioning; + self.cache = self + .cache + .with_partitioning(Self::output_partitioning_helper( + &self.input, + self.preserve_partitioning, + )); self } @@ -161,27 +168,38 @@ impl PartialSortExec { self.fetch } - fn with_cache(mut self) -> Self { + fn output_partitioning_helper( + input: &Arc, + preserve_partitioning: bool, + ) -> Partitioning { + // Get output partitioning: + if preserve_partitioning { + input.output_partitioning().clone() + } else { + Partitioning::UnknownPartitioning(1) + } + } + + fn create_cache( + input: &Arc, + sort_exprs: LexOrdering, + preserve_partitioning: bool, + ) -> PlanPropertiesCache { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: - let eq_properties = self - .input + let eq_properties = input .equivalence_properties() .clone() - .with_reorder(self.expr.to_vec()); + .with_reorder(sort_exprs); // Get output partitioning: - let output_partitioning = if self.preserve_partitioning { - self.input.output_partitioning().clone() - } else { - Partitioning::UnknownPartitioning(1) - }; + let output_partitioning = + Self::output_partitioning_helper(input, preserve_partitioning); // Determine execution mode: - let mode = self.input.execution_mode(); + let mode = input.execution_mode(); - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 55a3c9f068f1..a74705dd32ab 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -52,6 +52,7 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; +use datafusion_physical_expr::LexOrdering; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; @@ -694,16 +695,16 @@ impl SortExec { /// Create a new sort execution plan that produces a single, /// sorted output partition. pub fn new(expr: Vec, input: Arc) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); + let preserve_partitioning = false; + let cache = Self::create_cache(&input, expr.clone(), preserve_partitioning); Self { expr, input, metrics_set: ExecutionPlanMetricsSet::new(), - preserve_partitioning: false, + preserve_partitioning, fetch: None, cache, } - .with_cache() } /// Create a new sort execution plan with the option to preserve @@ -737,7 +738,13 @@ impl SortExec { /// input partitions producing a single, sorted partition. pub fn with_preserve_partitioning(mut self, preserve_partitioning: bool) -> Self { self.preserve_partitioning = preserve_partitioning; - self.with_cache() + self.cache = self + .cache + .with_partitioning(Self::output_partitioning_helper( + &self.input, + self.preserve_partitioning, + )); + self } /// Modify how many rows to include in the result @@ -767,33 +774,43 @@ impl SortExec { self.fetch } - fn with_cache(mut self) -> Self { + fn output_partitioning_helper( + input: &Arc, + preserve_partitioning: bool, + ) -> Partitioning { + // Get output partitioning: + if preserve_partitioning { + input.output_partitioning().clone() + } else { + Partitioning::UnknownPartitioning(1) + } + } + + fn create_cache( + input: &Arc, + sort_exprs: LexOrdering, + preserve_partitioning: bool, + ) -> PlanPropertiesCache { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: - let eq_properties = self - .input + let eq_properties = input .equivalence_properties() .clone() - .with_reorder(self.expr.to_vec()); + .with_reorder(sort_exprs); // Get output partitioning: - let output_partitioning = if self.preserve_partitioning { - self.input.output_partitioning().clone() - } else { - Partitioning::UnknownPartitioning(1) - }; + let output_partitioning = + Self::output_partitioning_helper(input, preserve_partitioning); // Determine execution mode: - let mode = match self.input.execution_mode() { + let mode = match input.execution_mode() { ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { ExecutionMode::PipelineBreaking } ExecutionMode::Bounded => ExecutionMode::Bounded, }; - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index eadd2d0711fe..c07ae72d5492 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -80,7 +80,7 @@ pub struct SortPreservingMergeExec { impl SortPreservingMergeExec { /// Create a new sort execution plan pub fn new(expr: Vec, input: Arc) -> Self { - let cache = PlanPropertiesCache::new_default(input.schema()); + let cache = Self::create_cache(&input); Self { input, expr, @@ -88,7 +88,6 @@ impl SortPreservingMergeExec { fetch: None, cache, } - .with_cache() } /// Sets the number of rows to fetch pub fn with_fetch(mut self, fetch: Option) -> Self { @@ -111,14 +110,12 @@ impl SortPreservingMergeExec { self.fetch } - fn with_cache(mut self) -> Self { - self.cache = PlanPropertiesCache::new( - self.input.equivalence_properties().clone(), // Equivalence Properties - Partitioning::UnknownPartitioning(1), // Output Partitioning - self.input.execution_mode(), // Execution Mode - ); - - self + fn create_cache(input: &Arc) -> PlanPropertiesCache { + PlanPropertiesCache::new( + input.equivalence_properties().clone(), // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + input.execution_mode(), // Execution Mode + ) } } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 60b372446805..e95fd37ab1b2 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -85,16 +85,22 @@ impl StreamingTableExec { Some(p) => Arc::new(schema.project(p)?), None => schema, }; - let cache = PlanPropertiesCache::new_default(projected_schema.clone()); + let projected_output_ordering = + projected_output_ordering.into_iter().collect::>(); + let cache = Self::create_cache( + projected_schema.clone(), + &projected_output_ordering, + &partitions, + infinite, + ); Ok(Self { partitions, projected_schema, projection: projection.cloned().map(Into::into), - projected_output_ordering: projected_output_ordering.into_iter().collect(), + projected_output_ordering, infinite, cache, - } - .with_cache()) + }) } pub fn partitions(&self) -> &Vec> { @@ -121,26 +127,26 @@ impl StreamingTableExec { self.infinite } - fn with_cache(mut self) -> Self { + fn create_cache( + schema: SchemaRef, + orderings: &[LexOrdering], + partitions: &[Arc], + is_infinite: bool, + ) -> PlanPropertiesCache { // Calculate equivalence properties: - let eq_properties = EquivalenceProperties::new_with_orderings( - self.schema(), - &self.projected_output_ordering, - ); + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); // Get output partitioning: - let output_partitioning = - Partitioning::UnknownPartitioning(self.partitions.len()); + let output_partitioning = Partitioning::UnknownPartitioning(partitions.len()); // Determine execution mode: - let mode = if self.infinite { + let mode = if is_infinite { ExecutionMode::Unbounded } else { ExecutionMode::Bounded }; - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 77ff8d27157a..a677907295a7 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -34,6 +34,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use futures::Stream; use tokio::sync::Barrier; @@ -132,14 +133,13 @@ impl MockExec { /// ensure any poll loops are correct. This behavior can be /// changed with `with_use_task` pub fn new(data: Vec>, schema: SchemaRef) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(schema.clone()); Self { data, schema, use_task: true, cache, } - .with_cache() } /// If `use_task` is true (the default) then the batches are sent @@ -150,15 +150,14 @@ impl MockExec { self } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } @@ -290,14 +289,13 @@ impl BarrierExec { pub fn new(data: Vec>, schema: SchemaRef) -> Self { // wait for all streams and the input let barrier = Arc::new(Barrier::new(data.len() + 1)); - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(schema.clone(), &data); Self { data, schema, barrier, cache, } - .with_cache() } /// wait until all the input streams and this function is ready @@ -307,15 +305,13 @@ impl BarrierExec { println!("BarrierExec::wait done waiting"); } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(self.data.len())) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + fn create_cache(schema: SchemaRef, data: &[Vec]) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(data.len()), + ExecutionMode::Bounded, + ) } } @@ -412,19 +408,18 @@ impl ErrorExec { DataType::Int64, true, )])); - let cache = PlanPropertiesCache::new_default(schema.clone()); - Self { cache }.with_cache() + let cache = Self::create_cache(schema.clone()); + Self { cache } } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(1)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } @@ -486,24 +481,22 @@ impl StatisticsExec { .column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); - let cache = PlanPropertiesCache::new_default(Arc::new(schema.clone())); + let cache = Self::create_cache(Arc::new(schema.clone())); Self { stats, schema: Arc::new(schema), cache, } - .with_cache() } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(2)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(2), + ExecutionMode::Bounded, + ) } } @@ -567,9 +560,6 @@ pub struct BlockingExec { /// Schema that is mocked by this plan. schema: SchemaRef, - /// Number of output partitions. - n_partitions: usize, - /// Ref-counting helper to check if the plan and the produced stream are still in memory. refs: Arc<()>, cache: PlanPropertiesCache, @@ -578,14 +568,12 @@ pub struct BlockingExec { impl BlockingExec { /// Create new [`BlockingExec`] with a give schema and number of partitions. pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(schema.clone(), n_partitions); Self { schema, - n_partitions, refs: Default::default(), cache, } - .with_cache() } /// Weak pointer that can be used for ref-counting this execution plan and its streams. @@ -597,15 +585,14 @@ impl BlockingExec { Arc::downgrade(&self.refs) } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(self.n_partitions)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(n_partitions), + ExecutionMode::Bounded, + ) } } @@ -719,13 +706,13 @@ impl PanicExec { /// Create new [`PanicExec`] with a give schema and number of /// partitions, which will each panic immediately. pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let batches_until_panics = vec![0; n_partitions]; + let cache = Self::create_cache(schema.clone(), &batches_until_panics); Self { schema, - batches_until_panics: vec![0; n_partitions], + batches_until_panics, cache, } - .with_cache() } /// Set the number of batches prior to panic for a partition @@ -734,17 +721,18 @@ impl PanicExec { self } - fn with_cache(mut self) -> Self { - let num_partitions = self.batches_until_panics.len(); + fn create_cache( + schema: SchemaRef, + batches_until_panics: &[usize], + ) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + let num_partitions = batches_until_panics.len(); - self.cache = self - .cache - // Output Partitioning - .with_partitioning(Partitioning::UnknownPartitioning(num_partitions)) - // Execution Mode - .with_exec_mode(ExecutionMode::Bounded); - - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(num_partitions), + ExecutionMode::Bounded, + ) } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 42e5ce58edb0..06a870123255 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -98,13 +98,12 @@ impl UnionExec { /// Create a new UnionExec pub fn new(inputs: Vec>) -> Self { let schema = union_schema(&inputs); - let cache = PlanPropertiesCache::new_default(schema); + let cache = Self::create_cache(&inputs, schema); UnionExec { inputs, metrics: ExecutionPlanMetricsSet::new(), cache, } - .with_cache() } /// Get inputs of the execution plan @@ -112,16 +111,18 @@ impl UnionExec { &self.inputs } - fn with_cache(mut self) -> Self { + fn create_cache( + inputs: &[Arc], + schema: SchemaRef, + ) -> PlanPropertiesCache { // Calculate equivalence properties: // TODO: In some cases, we should be able to preserve some equivalence // classes and constants. Add support for such cases. - let children_eqs = self - .inputs + let children_eqs = inputs .iter() .map(|child| child.equivalence_properties()) .collect::>(); - let mut eq_properties = EquivalenceProperties::new(self.schema()); + let mut eq_properties = EquivalenceProperties::new(schema); // Use the ordering equivalence class of the first child as the seed: let mut meets = children_eqs[0] .oeq_class() @@ -152,18 +153,16 @@ impl UnionExec { eq_properties.add_new_orderings(meets); // Calculate output partitioning; i.e. sum output partitions of the inputs. - let num_partitions = self - .inputs + let num_partitions = inputs .iter() .map(|plan| plan.output_partitioning().partition_count()) .sum(); let output_partitioning = Partitioning::UnknownPartitioning(num_partitions); // Determine execution mode: - let mode = exec_mode_flatten(self.inputs.iter()); + let mode = exec_mode_flatten(inputs.iter()); - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } @@ -323,20 +322,17 @@ pub struct InterleaveExec { impl InterleaveExec { /// Create a new InterleaveExec pub fn try_new(inputs: Vec>) -> Result { - let schema = union_schema(&inputs); - if !can_interleave(inputs.iter()) { return internal_err!( "Not all InterleaveExec children have a consistent hash partitioning" ); } - let cache = PlanPropertiesCache::new_default(schema); + let cache = Self::create_cache(&inputs); Ok(InterleaveExec { inputs, metrics: ExecutionPlanMetricsSet::new(), cache, - } - .with_cache()) + }) } /// Get inputs of the execution plan @@ -344,18 +340,15 @@ impl InterleaveExec { &self.inputs } - fn with_cache(mut self) -> Self { + fn create_cache(inputs: &[Arc]) -> PlanPropertiesCache { + let schema = union_schema(inputs); + let eq_properties = EquivalenceProperties::new(schema); // Get output partitioning: - let output_partitioning = self.inputs[0].output_partitioning().clone(); + let output_partitioning = inputs[0].output_partitioning().clone(); // Determine execution mode: - let mode = exec_mode_flatten(self.inputs.iter()); + let mode = exec_mode_flatten(inputs.iter()); - self.cache = self - .cache - .with_partitioning(output_partitioning) - .with_exec_mode(mode); - - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index b9df57d84f81..ba90e8b4f1fc 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -37,6 +37,7 @@ use arrow::datatypes::{ use arrow::record_batch::RecordBatch; use datafusion_common::{exec_err, DataFusionError, Result, UnnestOptions}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use futures::{Stream, StreamExt}; @@ -70,7 +71,7 @@ impl UnnestExec { schema: SchemaRef, options: UnnestOptions, ) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(&input, schema.clone()); UnnestExec { input, schema, @@ -79,18 +80,19 @@ impl UnnestExec { metrics: Default::default(), cache, } - .with_cache() } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - // Output Partitioning - .with_partitioning(self.input.output_partitioning().clone()) - // Execution Mode - .with_exec_mode(self.input.execution_mode()); - - self + fn create_cache( + input: &Arc, + schema: SchemaRef, + ) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); + + PlanPropertiesCache::new( + eq_properties, + input.output_partitioning().clone(), + input.execution_mode(), + ) } } diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 7fc242099379..20c8eddce6bd 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -33,6 +33,7 @@ use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::{internal_err, plan_err, DataFusionError, Result, ScalarValue}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; /// Execution plan for values list based relation (produces constant rows) #[derive(Debug)] @@ -113,13 +114,12 @@ impl ValuesExec { } } - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(schema.clone()); Ok(ValuesExec { schema, data: batches, cache, - } - .with_cache()) + }) } /// provides the data @@ -127,13 +127,14 @@ impl ValuesExec { self.data.clone() } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - .with_partitioning(Partitioning::UnknownPartitioning(1)) - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } 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 cb512302cb6f..a9dfc9bfeedd 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -121,8 +121,8 @@ impl BoundedWindowAggExec { vec![] } }; - let cache = PlanPropertiesCache::new_default(schema.clone()); - let window = Self { + let cache = Self::create_cache(&input, &schema, &window_expr); + Ok(Self { input, window_expr, schema, @@ -131,8 +131,7 @@ impl BoundedWindowAggExec { input_order_mode, ordered_partition_by_indices, cache, - }; - Ok(window.with_cache()) + }) } /// Window expressions @@ -183,23 +182,25 @@ impl BoundedWindowAggExec { }) } - fn with_cache(mut self) -> Self { + fn create_cache( + input: &Arc, + schema: &SchemaRef, + window_expr: &[Arc], + ) -> PlanPropertiesCache { // Calculate equivalence properties: - let eq_properties = - window_equivalence_properties(&self.schema, &self.input, &self.window_expr); + let eq_properties = window_equivalence_properties(schema, input, window_expr); // As we can have repartitioning using the partition keys, this can // be either one or more than one, depending on the presence of // repartitioning. - let output_partitioning = self.input.output_partitioning().clone(); + let output_partitioning = input.output_partitioning().clone(); // Construct properties cache - self.cache = PlanPropertiesCache::new( - eq_properties, // Equivalence Properties - output_partitioning, // Output Partitioning - self.input.execution_mode(), // Execution Mode - ); - self + PlanPropertiesCache::new( + eq_properties, // Equivalence Properties + output_partitioning, // Output Partitioning + input.execution_mode(), // Execution Mode + ) } } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 247588c971a2..852698bafe3a 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -80,8 +80,8 @@ impl WindowAggExec { let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); - let cache = PlanPropertiesCache::new_default(schema.clone()); - let window = Self { + let cache = Self::create_cache(schema.clone(), &input, &window_expr); + Ok(Self { input, window_expr, schema, @@ -89,8 +89,7 @@ impl WindowAggExec { metrics: ExecutionPlanMetricsSet::new(), ordered_partition_by_indices, cache, - }; - Ok(window.with_cache()) + }) } /// Window expressions @@ -117,18 +116,21 @@ impl WindowAggExec { ) } - fn with_cache(mut self) -> Self { + fn create_cache( + schema: SchemaRef, + input: &Arc, + window_expr: &[Arc], + ) -> PlanPropertiesCache { // Calculate equivalence properties: - let eq_properties = - window_equivalence_properties(&self.schema, &self.input, &self.window_expr); + let eq_properties = window_equivalence_properties(&schema, input, window_expr); // Get output partitioning: // Because we can have repartitioning using the partition keys this // would be either 1 or more than 1 depending on the presense of repartitioning. - let output_partitioning = self.input.output_partitioning().clone(); + let output_partitioning = input.output_partitioning().clone(); // Determine execution mode: - let mode = match self.input.execution_mode() { + let mode = match input.execution_mode() { ExecutionMode::Bounded => ExecutionMode::Bounded, ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { ExecutionMode::PipelineBreaking @@ -136,8 +138,7 @@ impl WindowAggExec { }; // Construct properties cache: - self.cache = PlanPropertiesCache::new(eq_properties, output_partitioning, mode); - self + PlanPropertiesCache::new(eq_properties, output_partitioning, mode) } } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index 33c611dd30d8..9a0b5daf27e4 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -33,7 +33,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::Partitioning; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; /// The name is from PostgreSQL's terminology. /// See @@ -91,7 +91,7 @@ pub struct WorkTableExec { impl WorkTableExec { /// Create a new execution plan for a worktable exec. pub fn new(name: String, schema: SchemaRef) -> Self { - let cache = PlanPropertiesCache::new_default(schema.clone()); + let cache = Self::create_cache(schema.clone()); Self { name, schema, @@ -99,7 +99,6 @@ impl WorkTableExec { work_table: Arc::new(WorkTable::new()), cache, } - .with_cache() } pub(super) fn with_work_table(&self, work_table: Arc) -> Self { @@ -112,13 +111,14 @@ impl WorkTableExec { } } - fn with_cache(mut self) -> Self { - self.cache = self - .cache - .with_partitioning(Partitioning::UnknownPartitioning(1)) - .with_exec_mode(ExecutionMode::Bounded); + fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + let eq_properties = EquivalenceProperties::new(schema); - self + PlanPropertiesCache::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } From 07a438d72b7e962fed92cb041f3063187e3cea29 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 27 Feb 2024 09:22:49 +0300 Subject: [PATCH 07/15] Address reviews --- datafusion-examples/examples/custom_datasource.rs | 1 + .../core/src/datasource/physical_plan/arrow_file.rs | 1 + datafusion/core/src/datasource/physical_plan/avro.rs | 1 + datafusion/core/src/datasource/physical_plan/csv.rs | 1 + datafusion/core/src/datasource/physical_plan/json.rs | 1 + .../core/src/datasource/physical_plan/parquet/mod.rs | 1 + .../core/src/physical_optimizer/enforce_distribution.rs | 1 + .../core/src/physical_optimizer/output_requirements.rs | 1 + datafusion/core/src/physical_planner.rs | 1 + datafusion/core/src/test/mod.rs | 1 + datafusion/core/src/test_util/mod.rs | 3 ++- datafusion/core/tests/custom_sources.rs | 1 + .../custom_sources_cases/provider_filter_pushdown.rs | 1 + datafusion/core/tests/custom_sources_cases/statistics.rs | 1 + datafusion/core/tests/user_defined/user_defined_plan.rs | 3 ++- datafusion/physical-plan/src/aggregates/mod.rs | 2 ++ datafusion/physical-plan/src/analyze.rs | 1 + datafusion/physical-plan/src/coalesce_batches.rs | 1 + datafusion/physical-plan/src/coalesce_partitions.rs | 1 + datafusion/physical-plan/src/empty.rs | 1 + datafusion/physical-plan/src/explain.rs | 1 + datafusion/physical-plan/src/filter.rs | 2 ++ datafusion/physical-plan/src/joins/cross_join.rs | 1 + datafusion/physical-plan/src/joins/hash_join.rs | 4 ++-- datafusion/physical-plan/src/joins/nested_loop_join.rs | 1 + datafusion/physical-plan/src/joins/sort_merge_join.rs | 1 + .../physical-plan/src/joins/symmetric_hash_join.rs | 1 + datafusion/physical-plan/src/limit.rs | 2 ++ datafusion/physical-plan/src/memory.rs | 1 + datafusion/physical-plan/src/placeholder_row.rs | 1 + datafusion/physical-plan/src/projection.rs | 1 + datafusion/physical-plan/src/recursive_query.rs | 1 + datafusion/physical-plan/src/repartition/mod.rs | 9 +++------ datafusion/physical-plan/src/sorts/partial_sort.rs | 1 + datafusion/physical-plan/src/sorts/sort.rs | 1 + .../physical-plan/src/sorts/sort_preserving_merge.rs | 1 + datafusion/physical-plan/src/streaming.rs | 1 + datafusion/physical-plan/src/test/exec.rs | 6 ++++++ datafusion/physical-plan/src/union.rs | 2 ++ datafusion/physical-plan/src/unnest.rs | 1 + datafusion/physical-plan/src/values.rs | 1 + .../physical-plan/src/windows/bounded_window_agg_exec.rs | 1 + datafusion/physical-plan/src/windows/window_agg_exec.rs | 1 + datafusion/physical-plan/src/work_table.rs | 1 + 44 files changed, 58 insertions(+), 10 deletions(-) diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index d3cd66b2c9bc..1ce3ced0e1c4 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -208,6 +208,7 @@ impl CustomExec { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); PlanPropertiesCache::new( diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 24e825a6920b..8eebc2b68f8b 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -84,6 +84,7 @@ impl ArrowExec { 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 create_cache( schema: SchemaRef, projected_output_ordering: &[LexOrdering], diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 6b6e7bce90c1..9d65a0ce089a 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -69,6 +69,7 @@ impl AvroExec { &self.base_config } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( schema: SchemaRef, orderings: &[LexOrdering], diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 19281bc3c189..964f40b8e002 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -121,6 +121,7 @@ impl CsvExec { 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 create_cache( schema: SchemaRef, orderings: &[LexOrdering], diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 6e17e58d8444..b27bcdaa917c 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 { 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 create_cache( schema: SchemaRef, orderings: &[LexOrdering], diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 810a84646c86..e2ad1980d422 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -266,6 +266,7 @@ impl ParquetExec { 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 create_cache( schema: SchemaRef, orderings: &[LexOrdering], diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index a5ad2d546d41..86a490278b0e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1360,6 +1360,7 @@ pub(crate) mod tests { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(input: &Arc) -> PlanPropertiesCache { PlanPropertiesCache::new( input.equivalence_properties().clone(), // Equivalence Properties diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index a806580ce716..129ae538808f 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -112,6 +112,7 @@ impl OutputRequirementExec { self.input.clone() } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(input: &Arc) -> PlanPropertiesCache { PlanPropertiesCache::new( input.equivalence_properties().clone(), // Equivalence Properties diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index e571bc76f4d5..8049c3940a1a 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2584,6 +2584,7 @@ mod tests { Self { cache } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); PlanPropertiesCache::new( diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index f8eb67cfdaf5..e5d8f6ebda32 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -384,6 +384,7 @@ impl StatisticsExec { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); PlanPropertiesCache::new( diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index dda6d730ce84..55a30b07d893 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -46,6 +46,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::TableReference; use datafusion_expr::{CreateExternalTable, Expr, TableType}; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use futures::Stream; @@ -55,7 +56,6 @@ use tempfile::TempDir; #[cfg(feature = "parquet")] pub use datafusion_common::test_util::parquet_test_data; pub use datafusion_common::test_util::{arrow_test_data, get_data_dir}; -use datafusion_physical_expr::EquivalenceProperties; /// Scan an empty data source, mainly used in tests pub fn scan_empty( @@ -246,6 +246,7 @@ impl UnboundedExec { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( schema: SchemaRef, batch_produce: Option, diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 11f29192276c..f62a3f723ad7 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -86,6 +86,7 @@ impl CustomExecutionPlan { Self { projection, cache } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); PlanPropertiesCache::new( diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index da00effa00a8..dec2deb10cbb 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -67,6 +67,7 @@ impl CustomPlan { Self { batches, cache } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); PlanPropertiesCache::new( diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 37854908f021..e98781aae9bf 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -61,6 +61,7 @@ impl StatisticsValidation { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index f2b6f6c93615..5f01334a4757 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -81,6 +81,7 @@ use datafusion::{ UserDefinedLogicalNodeCore, }, optimizer::{optimize_children, OptimizerConfig, OptimizerRule}, + physical_expr::EquivalenceProperties, physical_plan::{ DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, Partitioning, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, @@ -91,7 +92,6 @@ use datafusion::{ }; use async_trait::async_trait; -use datafusion_physical_expr::EquivalenceProperties; use futures::{Stream, StreamExt}; /// Execute the specified sql and return the resulting record batches @@ -421,6 +421,7 @@ impl TopKExec { Self { input, k, cache } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index fa5b65e40123..35e42b8a4d36 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -506,6 +506,7 @@ impl AggregateExec { true } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, schema: SchemaRef, @@ -1629,6 +1630,7 @@ mod tests { Self { yield_first, cache } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); PlanPropertiesCache::new( diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 731f3e3c7ebf..94dd6ff3bd28 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -81,6 +81,7 @@ impl AnalyzeExec { &self.input } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, schema: SchemaRef, diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index e83bce0664a3..bce48698a558 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -73,6 +73,7 @@ impl CoalesceBatchesExec { self.target_batch_size } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(input: &Arc) -> PlanPropertiesCache { // The coalesce batches operator does not make any changes to the // partitioning of its input. diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 27f58c9bfd85..ad1094cee0e1 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -57,6 +57,7 @@ impl CoalescePartitionsExec { &self.input } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(input: &Arc) -> PlanPropertiesCache { // Coalescing partitions loses existing orderings: let mut eq_properties = input.equivalence_properties().clone(); diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 942bee81f472..0705c4b4eca7 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -72,6 +72,7 @@ impl EmptyExec { Partitioning::UnknownPartitioning(n_partitions) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); let output_partitioning = Self::output_partitioning_helper(n_partitions); diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 689ef32aa1a9..200ba0bd07c5 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -72,6 +72,7 @@ impl ExplainExec { self.verbose } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); PlanPropertiesCache::new( diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index d6942f0d5678..86502039c8ba 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -115,6 +115,7 @@ impl FilterExec { self.default_selectivity } + /// Calculates `Statistics` for `FilterExec`, by applying selectivity (either default, or estimated) to input statistics. fn statistics_helper( input: &Arc, predicate: &Arc, @@ -157,6 +158,7 @@ impl FilterExec { }) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, predicate: &Arc, diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 42758e635060..3f6b6ba5e2b6 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -98,6 +98,7 @@ impl CrossJoinExec { &self.right } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( left: &Arc, right: &Arc, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 2b88ec449a04..4b010e8c60c6 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -37,7 +37,7 @@ use crate::{ check_join_is_valid, estimate_join_statistics, get_final_indices_from_bit_map, need_produce_result_in_final, partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinHashMap, JoinHashMapOffset, - JoinHashMapType, JoinOn, StatefulStreamResult, + JoinHashMapType, JoinOn, JoinOnRef, StatefulStreamResult, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, @@ -65,7 +65,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::PhysicalExprRef; -use crate::joins::utils::JoinOnRef; use ahash::RandomState; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -406,6 +405,7 @@ impl HashJoinExec { JoinSide::Right } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( left: &Arc, right: &Arc, diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 89beac14816d..bbfc4c12f548 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -144,6 +144,7 @@ impl NestedLoopJoinExec { &self.join_type } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( left: &Arc, right: &Arc, diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index f7d754a99e0e..20bae468cc4a 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -200,6 +200,7 @@ impl SortMergeJoinExec { self.left.as_ref() } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( left: &Arc, right: &Arc, diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 4e07b10dd517..3eff026a176f 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -252,6 +252,7 @@ impl SymmetricHashJoinExec { }) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( left: &Arc, right: &Arc, diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index a4b924d71066..e678360dd471 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -82,6 +82,7 @@ impl GlobalLimitExec { self.fetch } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(input: &Arc) -> PlanPropertiesCache { PlanPropertiesCache::new( input.equivalence_properties().clone(), // Equivalence Properties @@ -292,6 +293,7 @@ impl LocalLimitExec { self.fetch } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(input: &Arc) -> PlanPropertiesCache { PlanPropertiesCache::new( input.equivalence_properties().clone(), // Equivalence Properties diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index f6039ee8b3ed..8bd4db0bd418 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -204,6 +204,7 @@ impl MemoryExec { self.schema.clone() } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( schema: SchemaRef, orderings: &[LexOrdering], diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 9d4204ddb589..3280522e152c 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -94,6 +94,7 @@ impl PlaceholderRowExec { Partitioning::UnknownPartitioning(n_partitions) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); // Get output partitioning: diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 7420cf58b5ce..2ed8095f256c 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -113,6 +113,7 @@ impl ProjectionExec { &self.input } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, projection_mapping: &ProjectionMapping, diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index adc675ba2730..fd0d506e2ce4 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -94,6 +94,7 @@ impl RecursiveQueryExec { }) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index dc1e88f52e56..b9489bd12e64 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -616,12 +616,8 @@ impl RepartitionExec { input: &Arc, preserve_order: bool, ) -> Vec { - if preserve_order { - vec![true] - } else { - // We preserve ordering when input partitioning is 1 - vec![input.output_partitioning().partition_count() <= 1] - } + // We preserve ordering when repartition is order preserving variant or input partitioning is 1 + vec![preserve_order || input.output_partitioning().partition_count() <= 1] } fn eq_properties_helper( @@ -637,6 +633,7 @@ impl RepartitionExec { eq_properties } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, partitioning: Partitioning, diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 16c4bc8601b6..095245a706ea 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -180,6 +180,7 @@ impl PartialSortExec { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, sort_exprs: LexOrdering, diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a74705dd32ab..713ff86a5072 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -786,6 +786,7 @@ impl SortExec { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, sort_exprs: LexOrdering, diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index c07ae72d5492..16bdecd0f384 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -110,6 +110,7 @@ impl SortPreservingMergeExec { self.fetch } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(input: &Arc) -> PlanPropertiesCache { PlanPropertiesCache::new( input.equivalence_properties().clone(), // Equivalence Properties diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index e95fd37ab1b2..90e8600d78e5 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -127,6 +127,7 @@ impl StreamingTableExec { self.infinite } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( schema: SchemaRef, orderings: &[LexOrdering], diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index a677907295a7..a70e05809923 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -150,6 +150,7 @@ impl MockExec { self } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); @@ -305,6 +306,7 @@ impl BarrierExec { println!("BarrierExec::wait done waiting"); } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef, data: &[Vec]) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); PlanPropertiesCache::new( @@ -412,6 +414,7 @@ impl ErrorExec { Self { cache } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); @@ -489,6 +492,7 @@ impl StatisticsExec { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); @@ -585,6 +589,7 @@ impl BlockingExec { Arc::downgrade(&self.refs) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); @@ -721,6 +726,7 @@ impl PanicExec { self } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( schema: SchemaRef, batches_until_panics: &[usize], diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 06a870123255..4d5377a9bdcc 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -111,6 +111,7 @@ impl UnionExec { &self.inputs } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( inputs: &[Arc], schema: SchemaRef, @@ -340,6 +341,7 @@ impl InterleaveExec { &self.inputs } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(inputs: &[Arc]) -> PlanPropertiesCache { let schema = union_schema(inputs); let eq_properties = EquivalenceProperties::new(schema); diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index ba90e8b4f1fc..d727091fd1c3 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -82,6 +82,7 @@ impl UnnestExec { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, schema: SchemaRef, diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 20c8eddce6bd..f31272879279 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -127,6 +127,7 @@ impl ValuesExec { self.data.clone() } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); 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 a9dfc9bfeedd..d7579cdc041d 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -182,6 +182,7 @@ impl BoundedWindowAggExec { }) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( input: &Arc, schema: &SchemaRef, diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 852698bafe3a..f143d228f381 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -116,6 +116,7 @@ impl WindowAggExec { ) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache( schema: SchemaRef, input: &Arc, diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index 9a0b5daf27e4..44a42a4fcf92 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -111,6 +111,7 @@ impl WorkTableExec { } } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { let eq_properties = EquivalenceProperties::new(schema); From eced5bc002b1a0442f55e43751973292f88d17d9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 27 Feb 2024 19:04:09 +0300 Subject: [PATCH 08/15] Address reviews --- .../examples/custom_datasource.rs | 12 ++-- .../datasource/physical_plan/arrow_file.rs | 14 ++-- .../core/src/datasource/physical_plan/avro.rs | 14 ++-- .../core/src/datasource/physical_plan/csv.rs | 14 ++-- .../core/src/datasource/physical_plan/json.rs | 14 ++-- .../datasource/physical_plan/parquet/mod.rs | 16 ++--- .../enforce_distribution.rs | 12 ++-- .../physical_optimizer/output_requirements.rs | 12 ++-- datafusion/core/src/physical_planner.rs | 12 ++-- datafusion/core/src/test/mod.rs | 12 ++-- datafusion/core/src/test_util/mod.rs | 14 ++-- datafusion/core/tests/custom_sources.rs | 12 ++-- .../provider_filter_pushdown.rs | 12 ++-- .../tests/custom_sources_cases/statistics.rs | 12 ++-- .../tests/user_defined/user_defined_plan.rs | 12 ++-- .../physical-plan/src/aggregates/mod.rs | 24 +++---- datafusion/physical-plan/src/analyze.rs | 14 ++-- .../physical-plan/src/coalesce_batches.rs | 12 ++-- .../physical-plan/src/coalesce_partitions.rs | 12 ++-- datafusion/physical-plan/src/display.rs | 4 +- datafusion/physical-plan/src/empty.rs | 12 ++-- datafusion/physical-plan/src/explain.rs | 12 ++-- datafusion/physical-plan/src/filter.rs | 15 +++-- datafusion/physical-plan/src/insert.rs | 10 +-- .../physical-plan/src/joins/cross_join.rs | 18 ++--- .../physical-plan/src/joins/hash_join.rs | 18 ++--- .../src/joins/nested_loop_join.rs | 19 +++--- .../src/joins/sort_merge_join.rs | 19 +++--- .../src/joins/symmetric_hash_join.rs | 19 +++--- datafusion/physical-plan/src/lib.rs | 20 +++--- datafusion/physical-plan/src/limit.rs | 22 +++--- datafusion/physical-plan/src/memory.rs | 14 ++-- .../physical-plan/src/placeholder_row.rs | 16 ++--- datafusion/physical-plan/src/projection.rs | 16 ++--- .../physical-plan/src/recursive_query.rs | 12 ++-- .../physical-plan/src/repartition/mod.rs | 17 +++-- .../physical-plan/src/sorts/partial_sort.rs | 14 ++-- datafusion/physical-plan/src/sorts/sort.rs | 15 ++--- .../src/sorts/sort_preserving_merge.rs | 12 ++-- datafusion/physical-plan/src/streaming.rs | 14 ++-- datafusion/physical-plan/src/test/exec.rs | 67 ++++++++++--------- datafusion/physical-plan/src/union.rs | 30 ++++----- datafusion/physical-plan/src/unnest.rs | 14 ++-- datafusion/physical-plan/src/values.rs | 12 ++-- .../src/windows/bounded_window_agg_exec.rs | 14 ++-- .../src/windows/window_agg_exec.rs | 14 ++-- datafusion/physical-plan/src/work_table.rs | 14 ++-- 47 files changed, 369 insertions(+), 370 deletions(-) diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 1ce3ced0e1c4..0b7e3d4c6442 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -31,7 +31,7 @@ use datafusion::execution::context::{SessionState, TaskContext}; use datafusion::physical_plan::memory::MemoryStream; use datafusion::physical_plan::{ project_schema, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, - Partitioning, PlanPropertiesCache, SendableRecordBatchStream, + Partitioning, PlanProperties, SendableRecordBatchStream, }; use datafusion::prelude::*; use datafusion_expr::{Expr, LogicalPlanBuilder}; @@ -190,7 +190,7 @@ impl TableProvider for CustomDataSource { struct CustomExec { db: CustomDataSource, projected_schema: SchemaRef, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl CustomExec { @@ -200,7 +200,7 @@ impl CustomExec { db: CustomDataSource, ) -> Self { let projected_schema = project_schema(&schema, projections).unwrap(); - let cache = Self::create_cache(projected_schema.clone()); + let cache = Self::compute_properties(projected_schema.clone()); Self { db, projected_schema, @@ -209,9 +209,9 @@ impl CustomExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -230,7 +230,7 @@ impl ExecutionPlan for CustomExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 8eebc2b68f8b..8f010f1dcbf8 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -37,7 +37,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::Statistics; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; +use datafusion_physical_plan::{ExecutionMode, PlanProperties}; use futures::StreamExt; use itertools::Itertools; @@ -53,7 +53,7 @@ pub struct ArrowExec { projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl ArrowExec { @@ -61,7 +61,7 @@ impl ArrowExec { pub fn new(base_config: FileScanConfig) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = Self::create_cache( + let cache = Self::compute_properties( projected_schema.clone(), &projected_output_ordering, &base_config, @@ -85,16 +85,16 @@ impl ArrowExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, projected_output_ordering: &[LexOrdering], file_scan_config: &FileScanConfig, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Equivalence Properties let eq_properties = EquivalenceProperties::new_with_orderings(schema, projected_output_ordering); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Self::output_partitioning_helper(file_scan_config), // Output Partitioning ExecutionMode::Bounded, // Execution Mode @@ -126,7 +126,7 @@ impl ExecutionPlan for ArrowExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 9d65a0ce089a..2b913d862576 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -25,7 +25,7 @@ use crate::error::Result; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, SendableRecordBatchStream, Statistics, + PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::datatypes::SchemaRef; @@ -42,7 +42,7 @@ pub struct AvroExec { projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl AvroExec { @@ -50,7 +50,7 @@ impl AvroExec { pub fn new(base_config: FileScanConfig) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = Self::create_cache( + let cache = Self::compute_properties( projected_schema.clone(), &projected_output_ordering, &base_config, @@ -70,16 +70,16 @@ impl AvroExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, orderings: &[LexOrdering], file_scan_config: &FileScanConfig, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Equivalence Properties let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); let n_partitions = file_scan_config.file_groups.len(); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(n_partitions), // Output Partitioning ExecutionMode::Bounded, // Execution Mode @@ -103,7 +103,7 @@ impl ExecutionPlan for AvroExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 964f40b8e002..a509121a82c8 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -33,7 +33,7 @@ use crate::error::{DataFusionError, Result}; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, SendableRecordBatchStream, Statistics, + PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::csv; @@ -61,7 +61,7 @@ pub struct CsvExec { metrics: ExecutionPlanMetricsSet, /// Compression type of the file associated with CsvExec pub file_compression_type: FileCompressionType, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl CsvExec { @@ -76,7 +76,7 @@ impl CsvExec { ) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = Self::create_cache( + let cache = Self::compute_properties( projected_schema, &projected_output_ordering, &base_config, @@ -122,15 +122,15 @@ impl CsvExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, orderings: &[LexOrdering], file_scan_config: &FileScanConfig, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Equivalence Properties let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Self::output_partitioning_helper(file_scan_config), // Output Partitioning ExecutionMode::Bounded, // Execution Mode @@ -164,7 +164,7 @@ impl ExecutionPlan for CsvExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index b27bcdaa917c..7b0e84c4410b 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -33,7 +33,7 @@ use crate::error::{DataFusionError, Result}; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, SendableRecordBatchStream, Statistics, + PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::json::ReaderBuilder; @@ -55,7 +55,7 @@ pub struct NdJsonExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, file_compression_type: FileCompressionType, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl NdJsonExec { @@ -66,7 +66,7 @@ impl NdJsonExec { ) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = Self::create_cache( + let cache = Self::compute_properties( projected_schema, &projected_output_ordering, &base_config, @@ -90,15 +90,15 @@ impl NdJsonExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, orderings: &[LexOrdering], file_scan_config: &FileScanConfig, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Equivalence Properties let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Self::output_partitioning_helper(file_scan_config), // Output Partitioning ExecutionMode::Bounded, // Execution Mode @@ -129,7 +129,7 @@ impl ExecutionPlan for NdJsonExec { fn as_any(&self) -> &dyn Any { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index e2ad1980d422..300ced3a7657 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -38,8 +38,8 @@ use crate::{ physical_optimizer::pruning::PruningPredicate, physical_plan::{ metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, SendableRecordBatchStream, Statistics, + DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + SendableRecordBatchStream, Statistics, }, }; @@ -100,7 +100,7 @@ pub struct ParquetExec { metadata_size_hint: Option, /// Optional user defined parquet file reader factory parquet_file_reader_factory: Option>, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl ParquetExec { @@ -148,7 +148,7 @@ impl ParquetExec { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - let cache = Self::create_cache( + let cache = Self::compute_properties( projected_schema, &projected_output_ordering, &base_config, @@ -267,15 +267,15 @@ impl ParquetExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, orderings: &[LexOrdering], file_config: &FileScanConfig, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Equivalence Properties let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Self::output_partitioning_helper(file_config), // Output Partitioning ExecutionMode::Bounded, // Execution Mode @@ -335,7 +335,7 @@ impl ExecutionPlan for ParquetExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 86a490278b0e..c7ffc7838b36 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1331,7 +1331,7 @@ pub(crate) mod tests { expressions, expressions::binary, expressions::lit, expressions::Column, LexOrdering, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; - use datafusion_physical_plan::PlanPropertiesCache; + use datafusion_physical_plan::PlanProperties; /// Models operators like BoundedWindowExec that require an input /// ordering but is easy to construct @@ -1339,7 +1339,7 @@ pub(crate) mod tests { struct SortRequiredExec { input: Arc, expr: LexOrdering, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl SortRequiredExec { @@ -1352,7 +1352,7 @@ pub(crate) mod tests { input: Arc, requirement: Vec, ) -> Self { - let cache = Self::create_cache(&input); + let cache = Self::compute_properties(&input); Self { input, expr: requirement, @@ -1361,8 +1361,8 @@ pub(crate) mod tests { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(input: &Arc) -> PlanPropertiesCache { - PlanPropertiesCache::new( + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( input.equivalence_properties().clone(), // Equivalence Properties input.output_partitioning().clone(), // Output Partitioning input.execution_mode(), // Execution Mode @@ -1389,7 +1389,7 @@ pub(crate) mod tests { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 129ae538808f..992a6e7f82c0 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -33,7 +33,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion_physical_plan::PlanPropertiesCache; +use datafusion_physical_plan::PlanProperties; /// This rule either adds or removes [`OutputRequirements`]s to/from the physical /// plan according to its `mode` attribute, which is set by the constructors @@ -90,7 +90,7 @@ pub(crate) struct OutputRequirementExec { input: Arc, order_requirement: Option, dist_requirement: Distribution, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl OutputRequirementExec { @@ -99,7 +99,7 @@ impl OutputRequirementExec { requirements: Option, dist_requirement: Distribution, ) -> Self { - let cache = Self::create_cache(&input); + let cache = Self::compute_properties(&input); Self { input, order_requirement: requirements, @@ -113,8 +113,8 @@ impl OutputRequirementExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(input: &Arc) -> PlanPropertiesCache { - PlanPropertiesCache::new( + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( input.equivalence_properties().clone(), // Equivalence Properties input.output_partitioning().clone(), // Output Partitioning input.execution_mode(), // Execution Mode @@ -137,7 +137,7 @@ impl ExecutionPlan for OutputRequirementExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 8049c3940a1a..7e858953ac8f 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1995,7 +1995,7 @@ mod tests { use crate::datasource::MemTable; use crate::physical_plan::{ expressions, DisplayAs, DisplayFormatType, ExecutionMode, Partitioning, - PlanPropertiesCache, SendableRecordBatchStream, + PlanProperties, SendableRecordBatchStream, }; use crate::physical_planner::PhysicalPlanner; use crate::prelude::{SessionConfig, SessionContext}; @@ -2575,19 +2575,19 @@ mod tests { #[derive(Debug)] struct NoOpExecutionPlan { - cache: PlanPropertiesCache, + cache: PlanProperties, } impl NoOpExecutionPlan { fn new(schema: SchemaRef) -> Self { - let cache = Self::create_cache(schema.clone()); + let cache = Self::compute_properties(schema.clone()); Self { cache } } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Output Partitioning Partitioning::UnknownPartitioning(1), @@ -2613,7 +2613,7 @@ mod tests { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index e5d8f6ebda32..0042554f6c73 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -45,7 +45,7 @@ use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalSortExpr}; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionMode, PlanPropertiesCache, + DisplayAs, DisplayFormatType, ExecutionMode, PlanProperties, }; #[cfg(feature = "compression")] @@ -367,7 +367,7 @@ pub fn csv_exec_ordered( pub struct StatisticsExec { stats: Statistics, schema: Arc, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl StatisticsExec { @@ -376,7 +376,7 @@ impl StatisticsExec { stats.column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); - let cache = Self::create_cache(Arc::new(schema.clone())); + let cache = Self::compute_properties(Arc::new(schema.clone())); Self { stats, schema: Arc::new(schema), @@ -385,9 +385,9 @@ impl StatisticsExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Output Partitioning Partitioning::UnknownPartitioning(2), @@ -421,7 +421,7 @@ impl ExecutionPlan for StatisticsExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 55a30b07d893..3244ad49d1c6 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -38,7 +38,7 @@ use crate::execution::context::{SessionState, TaskContext}; use crate::logical_expr::{LogicalPlanBuilder, UNNAMED_TABLE}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, }; use crate::prelude::{CsvReadOptions, SessionContext}; @@ -227,7 +227,7 @@ impl TableProvider for TestTableProvider { pub struct UnboundedExec { batch_produce: Option, batch: RecordBatch, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl UnboundedExec { /// Create new exec that clones the given record batch to its output. @@ -238,7 +238,7 @@ impl UnboundedExec { batch: RecordBatch, partitions: usize, ) -> Self { - let cache = Self::create_cache(batch.schema(), batch_produce, partitions); + let cache = Self::compute_properties(batch.schema(), batch_produce, partitions); Self { batch_produce, batch, @@ -247,18 +247,18 @@ impl UnboundedExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, batch_produce: Option, n_partitions: usize, - ) -> PlanPropertiesCache { + ) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); let mode = if batch_produce.is_none() { ExecutionMode::Unbounded } else { ExecutionMode::Bounded }; - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(n_partitions), mode, @@ -289,7 +289,7 @@ impl ExecutionPlan for UnboundedExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index f62a3f723ad7..aa3f35e29541 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -40,7 +40,7 @@ use datafusion_common::project_schema; use datafusion_common::stats::Precision; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; -use datafusion_physical_plan::{ExecutionMode, PlanPropertiesCache}; +use datafusion_physical_plan::{ExecutionMode, PlanProperties}; use async_trait::async_trait; use futures::stream::Stream; @@ -74,7 +74,7 @@ struct CustomTableProvider; #[derive(Debug, Clone)] struct CustomExecutionPlan { projection: Option>, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl CustomExecutionPlan { @@ -82,14 +82,14 @@ impl CustomExecutionPlan { let schema = TEST_CUSTOM_SCHEMA_REF!(); let schema = project_schema(&schema, projection.as_ref()).expect("projected schema"); - let cache = Self::create_cache(schema); + let cache = Self::compute_properties(schema); Self { projection, cache } } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Output Partitioning Partitioning::UnknownPartitioning(1), @@ -144,7 +144,7 @@ impl ExecutionPlan for CustomExecutionPlan { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index dec2deb10cbb..9f6c44d4603f 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -28,7 +28,7 @@ use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, SendableRecordBatchStream, Statistics, + PlanProperties, SendableRecordBatchStream, Statistics, }; use datafusion::prelude::*; use datafusion::scalar::ScalarValue; @@ -58,19 +58,19 @@ fn create_batch(value: i32, num_rows: usize) -> Result { #[derive(Debug)] struct CustomPlan { batches: Vec, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl CustomPlan { fn new(schema: SchemaRef, batches: Vec) -> Self { - let cache = Self::create_cache(schema); + let cache = Self::compute_properties(schema); Self { batches, cache } } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -97,7 +97,7 @@ impl ExecutionPlan for CustomPlan { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index e98781aae9bf..85ac47dc97fc 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -27,7 +27,7 @@ use datafusion::{ logical_expr::Expr, physical_plan::{ ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, - Partitioning, PlanPropertiesCache, SendableRecordBatchStream, Statistics, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }, prelude::SessionContext, scalar::ScalarValue, @@ -43,7 +43,7 @@ use async_trait::async_trait; struct StatisticsValidation { stats: Statistics, schema: Arc, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl StatisticsValidation { @@ -53,7 +53,7 @@ impl StatisticsValidation { schema.fields().len(), "the column statistics vector length should be the number of fields" ); - let cache = Self::create_cache(schema.clone()); + let cache = Self::compute_properties(schema.clone()); Self { stats, schema, @@ -62,10 +62,10 @@ impl StatisticsValidation { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(2), ExecutionMode::Bounded, @@ -149,7 +149,7 @@ impl ExecutionPlan for StatisticsValidation { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 5f01334a4757..2c12e108bb47 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -84,7 +84,7 @@ use datafusion::{ physical_expr::EquivalenceProperties, physical_plan::{ DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, - Partitioning, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }, physical_planner::{DefaultPhysicalPlanner, ExtensionPlanner, PhysicalPlanner}, @@ -412,20 +412,20 @@ struct TopKExec { input: Arc, /// The maxium number of values k: usize, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl TopKExec { fn new(input: Arc, k: usize) -> Self { - let cache = Self::create_cache(input.schema()); + let cache = Self::compute_properties(input.schema()); Self { input, k, cache } } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -460,7 +460,7 @@ impl ExecutionPlan for TopKExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 35e42b8a4d36..98d41cca6764 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use super::{DisplayAs, ExecutionMode, PlanPropertiesCache}; +use super::{DisplayAs, ExecutionMode, PlanProperties}; use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, @@ -265,7 +265,7 @@ pub struct AggregateExec { required_input_ordering: Option, /// Describes how the input is ordered relative to the group by columns input_order_mode: InputOrderMode, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl AggregateExec { @@ -362,7 +362,7 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - let cache = Self::create_cache( + let cache = Self::compute_properties( &input, schema.clone(), &projection_mapping, @@ -507,13 +507,13 @@ impl AggregateExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, schema: SchemaRef, projection_mapping: &ProjectionMapping, mode: &AggregateMode, input_order_mode: &InputOrderMode, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Construct equivalence properties: let eq_properties = input .equivalence_properties() @@ -550,7 +550,7 @@ impl AggregateExec { exec_mode = ExecutionMode::PipelineBreaking; } - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode) + PlanProperties::new(eq_properties, output_partitioning, exec_mode) } pub fn input_order_mode(&self) -> &InputOrderMode { @@ -641,7 +641,7 @@ impl ExecutionPlan for AggregateExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -1620,20 +1620,20 @@ mod tests { struct TestYieldingExec { /// True if this exec should yield back to runtime the first time it is polled pub yield_first: bool, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl TestYieldingExec { fn new(yield_first: bool) -> Self { let schema = some_data().0; - let cache = Self::create_cache(schema); + let cache = Self::compute_properties(schema); Self { yield_first, cache } } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Output Partitioning Partitioning::UnknownPartitioning(1), @@ -1662,7 +1662,7 @@ mod tests { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 94dd6ff3bd28..fed4b97d2afb 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use std::{any::Any, time::Instant}; use super::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; -use super::{DisplayAs, Distribution, PlanPropertiesCache, SendableRecordBatchStream}; +use super::{DisplayAs, Distribution, PlanProperties, SendableRecordBatchStream}; use crate::display::DisplayableExecutionPlan; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; @@ -45,7 +45,7 @@ pub struct AnalyzeExec { pub(crate) input: Arc, /// The output schema for RecordBatches of this exec node schema: SchemaRef, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl AnalyzeExec { @@ -56,7 +56,7 @@ impl AnalyzeExec { input: Arc, schema: SchemaRef, ) -> Self { - let cache = Self::create_cache(&input, schema.clone()); + let cache = Self::compute_properties(&input, schema.clone()); AnalyzeExec { verbose, show_statistics, @@ -82,14 +82,14 @@ impl AnalyzeExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, schema: SchemaRef, - ) -> PlanPropertiesCache { + ) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); let output_partitioning = Partitioning::UnknownPartitioning(1); let exec_mode = input.execution_mode(); - PlanPropertiesCache::new(eq_properties, output_partitioning, exec_mode) + PlanProperties::new(eq_properties, output_partitioning, exec_mode) } } @@ -113,7 +113,7 @@ impl ExecutionPlan for AnalyzeExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index bce48698a558..055f16288f95 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::{DisplayAs, PlanPropertiesCache, Statistics}; +use super::{DisplayAs, PlanProperties, Statistics}; use crate::{ DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; @@ -48,13 +48,13 @@ pub struct CoalesceBatchesExec { target_batch_size: usize, /// Execution metrics metrics: ExecutionPlanMetricsSet, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl CoalesceBatchesExec { /// Create a new CoalesceBatchesExec pub fn new(input: Arc, target_batch_size: usize) -> Self { - let cache = Self::create_cache(&input); + let cache = Self::compute_properties(&input); Self { input, target_batch_size, @@ -74,10 +74,10 @@ impl CoalesceBatchesExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(input: &Arc) -> PlanPropertiesCache { + fn compute_properties(input: &Arc) -> PlanProperties { // The coalesce batches operator does not make any changes to the // partitioning of its input. - PlanPropertiesCache::new( + PlanProperties::new( input.equivalence_properties().clone(), // Equivalence Properties input.output_partitioning().clone(), // Output Partitioning input.execution_mode(), // Execution Mode @@ -109,7 +109,7 @@ impl ExecutionPlan for CoalesceBatchesExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index ad1094cee0e1..7037445164a3 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::stream::{ObservedStream, RecordBatchReceiverStream}; -use super::{DisplayAs, PlanPropertiesCache, SendableRecordBatchStream, Statistics}; +use super::{DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; @@ -38,13 +38,13 @@ pub struct CoalescePartitionsExec { input: Arc, /// Execution metrics metrics: ExecutionPlanMetricsSet, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl CoalescePartitionsExec { /// Create a new CoalescePartitionsExec pub fn new(input: Arc) -> Self { - let cache = Self::create_cache(&input); + let cache = Self::compute_properties(&input); CoalescePartitionsExec { input, metrics: ExecutionPlanMetricsSet::new(), @@ -58,12 +58,12 @@ impl CoalescePartitionsExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(input: &Arc) -> PlanPropertiesCache { + fn compute_properties(input: &Arc) -> PlanProperties { // Coalescing partitions loses existing orderings: let mut eq_properties = input.equivalence_properties().clone(); eq_properties.clear_orderings(); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Equivalence Properties Partitioning::UnknownPartitioning(1), // Output Partitioning input.execution_mode(), // Execution Mode @@ -91,7 +91,7 @@ impl ExecutionPlan for CoalescePartitionsExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index d4afca6a1acf..38c23331983e 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -466,7 +466,7 @@ mod tests { use std::sync::Arc; use super::DisplayableExecutionPlan; - use crate::{DisplayAs, ExecutionPlan, PlanPropertiesCache}; + use crate::{DisplayAs, ExecutionPlan, PlanProperties}; use datafusion_common::{DataFusionError, Result, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -493,7 +493,7 @@ mod tests { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { unimplemented!() } diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 0705c4b4eca7..b6708515ec6b 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::sync::Arc; use super::{ - common, DisplayAs, ExecutionMode, PlanPropertiesCache, SendableRecordBatchStream, + common, DisplayAs, ExecutionMode, PlanProperties, SendableRecordBatchStream, Statistics, }; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; @@ -41,13 +41,13 @@ pub struct EmptyExec { schema: SchemaRef, /// Number of partitions partitions: usize, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl EmptyExec { /// Create a new EmptyExec pub fn new(schema: SchemaRef) -> Self { - let cache = Self::create_cache(schema.clone(), 1); + let cache = Self::compute_properties(schema.clone(), 1); EmptyExec { schema, partitions: 1, @@ -73,10 +73,10 @@ impl EmptyExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef, n_partitions: usize) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); let output_partitioning = Self::output_partitioning_helper(n_partitions); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Output Partitioning output_partitioning, @@ -106,7 +106,7 @@ impl ExecutionPlan for EmptyExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 200ba0bd07c5..f63b1df29da5 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use super::{DisplayAs, ExecutionMode, PlanPropertiesCache, SendableRecordBatchStream}; +use super::{DisplayAs, ExecutionMode, PlanProperties, SendableRecordBatchStream}; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; @@ -43,7 +43,7 @@ pub struct ExplainExec { stringified_plans: Vec, /// control which plans to print verbose: bool, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl ExplainExec { @@ -53,7 +53,7 @@ impl ExplainExec { stringified_plans: Vec, verbose: bool, ) -> Self { - let cache = Self::create_cache(schema.clone()); + let cache = Self::compute_properties(schema.clone()); ExplainExec { schema, stringified_plans, @@ -73,9 +73,9 @@ impl ExplainExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -103,7 +103,7 @@ impl ExecutionPlan for ExplainExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 86502039c8ba..95c09f541cc2 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::{ - ColumnStatistics, DisplayAs, PlanPropertiesCache, RecordBatchStream, + ColumnStatistics, DisplayAs, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::{ @@ -62,7 +62,7 @@ pub struct FilterExec { metrics: ExecutionPlanMetricsSet, /// Selectivity for statistics. 0 = no rows, 100 all rows default_selectivity: u8, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl FilterExec { @@ -74,7 +74,8 @@ impl FilterExec { match predicate.data_type(input.schema().as_ref())? { DataType::Boolean => { let default_selectivity = 20; - let cache = Self::create_cache(&input, &predicate, default_selectivity)?; + let cache = + Self::compute_properties(&input, &predicate, default_selectivity)?; Ok(Self { predicate, input: input.clone(), @@ -159,11 +160,11 @@ impl FilterExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, predicate: &Arc, default_selectivity: u8, - ) -> Result { + ) -> Result { // Combine the equal predicates with the input equivalence properties // to construct the equivalence properties: let stats = Self::statistics_helper(input, predicate, default_selectivity)?; @@ -182,7 +183,7 @@ impl FilterExec { .map(|column| Arc::new(column) as _); eq_properties = eq_properties.add_constants(constants); - Ok(PlanPropertiesCache::new( + Ok(PlanProperties::new( eq_properties, input.output_partitioning().clone(), // Output Partitioning input.execution_mode(), // Execution Mode @@ -210,7 +211,7 @@ impl ExecutionPlan for FilterExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 472c65f25b30..fd0bec108e03 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -23,7 +23,7 @@ use std::fmt::Debug; use std::sync::Arc; use super::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanPropertiesCache, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, }; use crate::metrics::MetricsSet; @@ -88,7 +88,7 @@ pub struct FileSinkExec { count_schema: SchemaRef, /// Optional required sort order for output data. sort_order: Option>, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl fmt::Debug for FileSinkExec { @@ -180,9 +180,9 @@ impl FileSinkExec { fn create_schema( input: &Arc, schema: SchemaRef, - ) -> PlanPropertiesCache { + ) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), input.execution_mode(), @@ -211,7 +211,7 @@ impl ExecutionPlan for FileSinkExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 3f6b6ba5e2b6..f73f3d36cdb4 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -27,8 +27,8 @@ use crate::coalesce_batches::concat_batches; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - exec_mode_flatten, ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, - ExecutionMode, ExecutionPlan, PlanPropertiesCache, RecordBatchStream, + execution_mode_from_children, ColumnStatistics, DisplayAs, DisplayFormatType, + Distribution, ExecutionMode, ExecutionPlan, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; @@ -61,7 +61,7 @@ pub struct CrossJoinExec { left_fut: OnceAsync, /// Execution plan metrics metrics: ExecutionPlanMetricsSet, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl CrossJoinExec { @@ -77,7 +77,7 @@ impl CrossJoinExec { }; let schema = Arc::new(Schema::new(all_columns)); - let cache = Self::create_cache(&left, &right, schema.clone()); + let cache = Self::compute_properties(&left, &right, schema.clone()); CrossJoinExec { left, right, @@ -99,11 +99,11 @@ impl CrossJoinExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( left: &Arc, right: &Arc, schema: SchemaRef, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties // TODO: Check equivalence properties of cross join, it may preserve // ordering in some cases. @@ -126,13 +126,13 @@ impl CrossJoinExec { ); // Determine the execution mode: - let mut mode = exec_mode_flatten([left, right]); + let mut mode = execution_mode_from_children([left, right]); if mode.is_unbounded() { // If any of the inputs is unbounded, cross join breaks the pipeline. mode = ExecutionMode::PipelineBreaking; } - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -197,7 +197,7 @@ impl ExecutionPlan for CrossJoinExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 4b010e8c60c6..2fdb2a17ebe8 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -29,7 +29,7 @@ use super::{ }; use crate::{ coalesce_partitions::CoalescePartitionsExec, - exec_mode_flatten, handle_state, + execution_mode_from_children, handle_state, hash_utils::create_hashes, joins::utils::{ adjust_indices_by_join_type, adjust_right_output_partitioning, @@ -41,7 +41,7 @@ use crate::{ }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, - Partitioning, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; @@ -297,7 +297,7 @@ pub struct HashJoinExec { /// matched and thus will not appear in the output. pub null_equals_null: bool, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl HashJoinExec { @@ -327,7 +327,7 @@ impl HashJoinExec { let random_state = RandomState::with_seeds(0, 0, 0, 0); - let cache = Self::create_cache( + let cache = Self::compute_properties( &left, &right, Arc::new(schema.clone()), @@ -406,14 +406,14 @@ impl HashJoinExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( left: &Arc, right: &Arc, schema: SchemaRef, join_type: JoinType, on: JoinOnRef, mode: PartitionMode, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( left.equivalence_properties().clone(), @@ -470,10 +470,10 @@ impl HashJoinExec { let mode = if pipeline_breaking { ExecutionMode::PipelineBreaking } else { - exec_mode_flatten([left, right]) + execution_mode_from_children([left, right]) }; - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -506,7 +506,7 @@ impl ExecutionPlan for HashJoinExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index bbfc4c12f548..5d2175d4a820 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -34,8 +34,9 @@ use crate::joins::utils::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - exec_mode_flatten, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, - ExecutionPlan, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + execution_mode_from_children, DisplayAs, DisplayFormatType, Distribution, + ExecutionMode, ExecutionPlan, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, }; use arrow::array::{ @@ -93,7 +94,7 @@ pub struct NestedLoopJoinExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl NestedLoopJoinExec { @@ -110,7 +111,7 @@ impl NestedLoopJoinExec { let (schema, column_indices) = build_join_schema(&left_schema, &right_schema, join_type); let schema = Arc::new(schema); - let cache = Self::create_cache(&left, &right, schema.clone(), *join_type); + let cache = Self::compute_properties(&left, &right, schema.clone(), *join_type); Ok(NestedLoopJoinExec { left, right, @@ -145,12 +146,12 @@ impl NestedLoopJoinExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( left: &Arc, right: &Arc, schema: SchemaRef, join_type: JoinType, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( left.equivalence_properties().clone(), @@ -176,12 +177,12 @@ impl NestedLoopJoinExec { }; // Determine execution mode: - let mut mode = exec_mode_flatten([left, right]); + let mut mode = execution_mode_from_children([left, right]); if mode.is_unbounded() { mode = ExecutionMode::PipelineBreaking; } - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -208,7 +209,7 @@ impl ExecutionPlan for NestedLoopJoinExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 20bae468cc4a..bde831b731ba 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -37,8 +37,8 @@ use crate::joins::utils::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ - exec_mode_flatten, metrics, DisplayAs, DisplayFormatType, Distribution, - ExecutionPlan, PhysicalExpr, PlanPropertiesCache, RecordBatchStream, + execution_mode_from_children, metrics, DisplayAs, DisplayFormatType, Distribution, + ExecutionPlan, PhysicalExpr, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; @@ -84,7 +84,7 @@ pub struct SortMergeJoinExec { /// If null_equals_null is true, null == null else null != null pub null_equals_null: bool, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl SortMergeJoinExec { @@ -137,7 +137,8 @@ impl SortMergeJoinExec { let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); - let cache = Self::create_cache(&left, &right, schema.clone(), join_type, &on); + let cache = + Self::compute_properties(&left, &right, schema.clone(), join_type, &on); Ok(Self { left, right, @@ -201,13 +202,13 @@ impl SortMergeJoinExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( left: &Arc, right: &Arc, schema: SchemaRef, join_type: JoinType, join_on: JoinOnRef, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( left.equivalence_properties().clone(), @@ -229,9 +230,9 @@ impl SortMergeJoinExec { ); // Determine execution mode: - let mode = exec_mode_flatten([left, right]); + let mode = execution_mode_from_children([left, right]); - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -265,7 +266,7 @@ impl ExecutionPlan for SortMergeJoinExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 3eff026a176f..77871a8b5483 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -46,11 +46,11 @@ use crate::joins::utils::{ JoinHashMapType, JoinOn, JoinOnRef, StatefulStreamResult, }; use crate::{ - exec_mode_flatten, + execution_mode_from_children, expressions::PhysicalSortExpr, joins::StreamJoinPartitionMode, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, PlanPropertiesCache, + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; @@ -192,7 +192,7 @@ pub struct SymmetricHashJoinExec { /// Partition Mode mode: StreamJoinPartitionMode, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl SymmetricHashJoinExec { @@ -234,7 +234,8 @@ impl SymmetricHashJoinExec { // Initialize the random state for the join operation: let random_state = RandomState::with_seeds(0, 0, 0, 0); let schema = Arc::new(schema); - let cache = Self::create_cache(&left, &right, schema.clone(), *join_type, &on); + let cache = + Self::compute_properties(&left, &right, schema.clone(), *join_type, &on); Ok(SymmetricHashJoinExec { left, right, @@ -253,13 +254,13 @@ impl SymmetricHashJoinExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( left: &Arc, right: &Arc, schema: SchemaRef, join_type: JoinType, join_on: JoinOnRef, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties: let eq_properties = join_equivalence_properties( left.equivalence_properties().clone(), @@ -282,9 +283,9 @@ impl SymmetricHashJoinExec { ); // Determine execution mode: - let mode = exec_mode_flatten([left, right]); + let mode = execution_mode_from_children([left, right]); - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } /// left stream @@ -390,7 +391,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index f90bbf061d38..3145c999d6b5 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -122,22 +122,22 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Get the schema for this execution plan fn schema(&self) -> SchemaRef { - self.cache().schema().clone() + self.properties().schema().clone() } - fn cache(&self) -> &PlanPropertiesCache; + fn properties(&self) -> &PlanProperties; /// Specifies how the output of this `ExecutionPlan` is split into /// partitions. fn output_partitioning(&self) -> &Partitioning { - &self.cache().partitioning + &self.properties().partitioning } /// Specifies whether this plan generates an infinite stream of records. /// If the plan does not support pipelining, but its input(s) are /// infinite, returns an error to indicate this. fn execution_mode(&self) -> ExecutionMode { - self.cache().exec_mode + self.properties().exec_mode } /// If the output of this `ExecutionPlan` within each partition is sorted, @@ -151,7 +151,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// It is safe to return `None` here if your `ExecutionPlan` does not /// have any particular output order here fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.cache().output_ordering.as_deref() + self.properties().output_ordering.as_deref() } /// Specifies the data distribution requirements for all the @@ -230,7 +230,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// See also [`Self::maintains_input_order`] and [`Self::output_ordering`] /// for related concepts. fn equivalence_properties(&self) -> &EquivalenceProperties { - &self.cache().eq_properties + &self.properties().eq_properties } /// Get a list of children `ExecutionPlan`s that act as inputs to this plan. @@ -482,7 +482,7 @@ impl ExecutionMode { } /// Conservatively "combines" execution modes of a given collection of operators. -fn exec_mode_flatten<'a>( +fn execution_mode_from_children<'a>( children: impl IntoIterator>, ) -> ExecutionMode { let mut result = ExecutionMode::Bounded; @@ -506,12 +506,12 @@ fn exec_mode_flatten<'a>( result } -/// Represents a cache for plan properties used in query optimization. +/// Stores the plan properties used in query optimization. /// /// This struct holds various properties useful for the query planning, which are used /// during optimization and execution phases. #[derive(Debug, Clone)] -pub struct PlanPropertiesCache { +pub struct PlanProperties { /// Stores the [`EquivalenceProperties`] of the [`ExecutionPlan`]. pub eq_properties: EquivalenceProperties, /// Stores the output [`Partitioning`] of the [`ExecutionPlan`]. @@ -523,7 +523,7 @@ pub struct PlanPropertiesCache { output_ordering: Option, } -impl PlanPropertiesCache { +impl PlanProperties { /// Construct a new `PlanPropertiesCache` from the pub fn new( eq_properties: EquivalenceProperties, diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index e678360dd471..a0b49d4ef136 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -24,7 +24,7 @@ use std::task::{Context, Poll}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{ - DisplayAs, ExecutionMode, PlanPropertiesCache, RecordBatchStream, + DisplayAs, ExecutionMode, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::{DisplayFormatType, Distribution, ExecutionPlan, Partitioning}; @@ -51,13 +51,13 @@ pub struct GlobalLimitExec { fetch: Option, /// Execution metrics metrics: ExecutionPlanMetricsSet, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl GlobalLimitExec { /// Create a new GlobalLimitExec pub fn new(input: Arc, skip: usize, fetch: Option) -> Self { - let cache = Self::create_cache(&input); + let cache = Self::compute_properties(&input); GlobalLimitExec { input, skip, @@ -83,8 +83,8 @@ impl GlobalLimitExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(input: &Arc) -> PlanPropertiesCache { - PlanPropertiesCache::new( + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( input.equivalence_properties().clone(), // Equivalence Properties Partitioning::UnknownPartitioning(1), // Output Partitioning ExecutionMode::Bounded, // Execution Mode @@ -117,7 +117,7 @@ impl ExecutionPlan for GlobalLimitExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -268,13 +268,13 @@ pub struct LocalLimitExec { fetch: usize, /// Execution metrics metrics: ExecutionPlanMetricsSet, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl LocalLimitExec { /// Create a new LocalLimitExec partition pub fn new(input: Arc, fetch: usize) -> Self { - let cache = Self::create_cache(&input); + let cache = Self::compute_properties(&input); Self { input, fetch, @@ -294,8 +294,8 @@ impl LocalLimitExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(input: &Arc) -> PlanPropertiesCache { - PlanPropertiesCache::new( + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( input.equivalence_properties().clone(), // Equivalence Properties input.output_partitioning().clone(), // Output Partitioning ExecutionMode::Bounded, // Execution Mode @@ -323,7 +323,7 @@ impl ExecutionPlan for LocalLimitExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 8bd4db0bd418..23699295e121 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -25,7 +25,7 @@ use std::task::{Context, Poll}; use super::expressions::PhysicalSortExpr; use super::{ common, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::datatypes::SchemaRef; @@ -48,7 +48,7 @@ pub struct MemoryExec { projection: Option>, // Sort information: one or more equivalent orderings sort_information: Vec, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl fmt::Debug for MemoryExec { @@ -101,7 +101,7 @@ impl ExecutionPlan for MemoryExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -153,7 +153,7 @@ impl MemoryExec { projection: Option>, ) -> Result { let projected_schema = project_schema(&schema, projection.as_ref())?; - let cache = Self::create_cache(projected_schema.clone(), &[], partitions); + let cache = Self::compute_properties(projected_schema.clone(), &[], partitions); Ok(Self { partitions: partitions.to_vec(), schema, @@ -205,13 +205,13 @@ impl MemoryExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, orderings: &[LexOrdering], partitions: &[Vec], - ) -> PlanPropertiesCache { + ) -> PlanProperties { let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Equivalence Properties Partitioning::UnknownPartitioning(partitions.len()), // Output Partitioning ExecutionMode::Bounded, // Execution Mode diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 3280522e152c..3fc9a512e79e 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::sync::Arc; use super::{ - common, DisplayAs, ExecutionMode, PlanPropertiesCache, SendableRecordBatchStream, + common, DisplayAs, ExecutionMode, PlanProperties, SendableRecordBatchStream, Statistics, }; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; @@ -43,14 +43,14 @@ pub struct PlaceholderRowExec { schema: SchemaRef, /// Number of partitions partitions: usize, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl PlaceholderRowExec { /// Create a new PlaceholderRowExec pub fn new(schema: SchemaRef) -> Self { let partitions = 1; - let cache = Self::create_cache(schema.clone(), partitions); + let cache = Self::compute_properties(schema.clone(), partitions); PlaceholderRowExec { schema, partitions, @@ -95,16 +95,12 @@ impl PlaceholderRowExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef, n_partitions: usize) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); // Get output partitioning: let output_partitioning = Self::output_partitioning_helper(n_partitions); - PlanPropertiesCache::new( - eq_properties, - output_partitioning, - ExecutionMode::Bounded, - ) + PlanProperties::new(eq_properties, output_partitioning, ExecutionMode::Bounded) } } @@ -128,7 +124,7 @@ impl ExecutionPlan for PlaceholderRowExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 2ed8095f256c..402feeaf80ba 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,8 +29,7 @@ use std::task::{Context, Poll}; use super::expressions::Column; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{ - DisplayAs, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, - Statistics, + DisplayAs, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::{ ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, @@ -59,7 +58,7 @@ pub struct ProjectionExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl ProjectionExec { @@ -93,7 +92,8 @@ impl ProjectionExec { // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; - let cache = Self::create_cache(&input, &projection_mapping, schema.clone())?; + let cache = + Self::compute_properties(&input, &projection_mapping, schema.clone())?; Ok(Self { expr, schema, @@ -114,11 +114,11 @@ impl ProjectionExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, projection_mapping: &ProjectionMapping, schema: SchemaRef, - ) -> Result { + ) -> Result { // Calculate equivalence properties: let mut input_eq_properties = input.equivalence_properties().clone(); input_eq_properties.substitute_oeq_class(projection_mapping)?; @@ -143,7 +143,7 @@ impl ProjectionExec { input_partition.clone() }; - Ok(PlanPropertiesCache::new( + Ok(PlanProperties::new( eq_properties, output_partitioning, input.execution_mode(), @@ -184,7 +184,7 @@ impl ExecutionPlan for ProjectionExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index fd0d506e2ce4..9786b1cbf6fd 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -24,7 +24,7 @@ use std::task::{Context, Poll}; use super::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, work_table::{WorkTable, WorkTableExec}, - PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::{DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan}; @@ -67,7 +67,7 @@ pub struct RecursiveQueryExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl RecursiveQueryExec { @@ -82,7 +82,7 @@ impl RecursiveQueryExec { let work_table = Arc::new(WorkTable::new()); // Use the same work table for both the WorkTableExec and the recursive term let recursive_term = assign_work_table(recursive_term, work_table.clone())?; - let cache = Self::create_cache(static_term.schema()); + let cache = Self::compute_properties(static_term.schema()); Ok(RecursiveQueryExec { name, static_term, @@ -95,10 +95,10 @@ impl RecursiveQueryExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -111,7 +111,7 @@ impl ExecutionPlan for RecursiveQueryExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index b9489bd12e64..1b92a0b03e49 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -34,9 +34,7 @@ use crate::repartition::distributor_channels::{ channels, partition_aware_channels, DistributionReceiver, DistributionSender, }; use crate::sorts::streaming_merge; -use crate::{ - DisplayFormatType, ExecutionPlan, Partitioning, PlanPropertiesCache, Statistics, -}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; use arrow::array::{ArrayRef, UInt64Builder}; use arrow::datatypes::SchemaRef; @@ -305,7 +303,7 @@ pub struct RepartitionExec { /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. preserve_order: bool, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } #[derive(Debug, Clone)] @@ -411,7 +409,7 @@ impl ExecutionPlan for RepartitionExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -598,7 +596,8 @@ impl RepartitionExec { partitioning: Partitioning, ) -> Result { let preserve_order = false; - let cache = Self::create_cache(&input, partitioning.clone(), preserve_order); + let cache = + Self::compute_properties(&input, partitioning.clone(), preserve_order); Ok(RepartitionExec { input, partitioning, @@ -634,15 +633,15 @@ impl RepartitionExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, partitioning: Partitioning, preserve_order: bool, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Equivalence Properties let eq_properties = Self::eq_properties_helper(input, preserve_order); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Equivalence Properties partitioning, // Output Partitioning input.execution_mode(), // Execution Mode diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 095245a706ea..01e592d29f7b 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -62,7 +62,7 @@ use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::sort::sort_batch; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PlanPropertiesCache, SendableRecordBatchStream, Statistics, + PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::compute::concat_batches; @@ -94,7 +94,7 @@ pub struct PartialSortExec { /// Fetch highest/lowest n results fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl PartialSortExec { @@ -106,7 +106,7 @@ impl PartialSortExec { ) -> Self { assert!(common_prefix_length > 0); let preserve_partitioning = false; - let cache = Self::create_cache(&input, expr.clone(), preserve_partitioning); + let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning); Self { input, expr, @@ -181,11 +181,11 @@ impl PartialSortExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, sort_exprs: LexOrdering, preserve_partitioning: bool, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: let eq_properties = input @@ -200,7 +200,7 @@ impl PartialSortExec { // Determine execution mode: let mode = input.execution_mode(); - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -230,7 +230,7 @@ impl ExecutionPlan for PartialSortExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 713ff86a5072..7109d730f1dd 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -37,8 +37,7 @@ use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::topk::TopK; use crate::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionMode, - ExecutionPlan, Partitioning, PlanPropertiesCache, SendableRecordBatchStream, - Statistics, + ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::compute::{concat_batches, lexsort_to_indices, take}; @@ -678,7 +677,7 @@ pub struct SortExec { /// Fetch highest/lowest n results fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl SortExec { @@ -696,7 +695,7 @@ impl SortExec { /// sorted output partition. pub fn new(expr: Vec, input: Arc) -> Self { let preserve_partitioning = false; - let cache = Self::create_cache(&input, expr.clone(), preserve_partitioning); + let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning); Self { expr, input, @@ -787,11 +786,11 @@ impl SortExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, sort_exprs: LexOrdering, preserve_partitioning: bool, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: let eq_properties = input @@ -811,7 +810,7 @@ impl SortExec { ExecutionMode::Bounded => ExecutionMode::Bounded, }; - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -840,7 +839,7 @@ impl ExecutionPlan for SortExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 16bdecd0f384..862146e10549 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -26,7 +26,7 @@ use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::streaming_merge; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PlanPropertiesCache, SendableRecordBatchStream, Statistics, + PlanProperties, SendableRecordBatchStream, Statistics, }; use datafusion_common::{internal_err, DataFusionError, Result}; @@ -74,13 +74,13 @@ pub struct SortPreservingMergeExec { /// Optional number of rows to fetch. Stops producing rows after this fetch fetch: Option, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl SortPreservingMergeExec { /// Create a new sort execution plan pub fn new(expr: Vec, input: Arc) -> Self { - let cache = Self::create_cache(&input); + let cache = Self::compute_properties(&input); Self { input, expr, @@ -111,8 +111,8 @@ impl SortPreservingMergeExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(input: &Arc) -> PlanPropertiesCache { - PlanPropertiesCache::new( + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( input.equivalence_properties().clone(), // Equivalence Properties Partitioning::UnknownPartitioning(1), // Output Partitioning input.execution_mode(), // Execution Mode @@ -149,7 +149,7 @@ impl ExecutionPlan for SortPreservingMergeExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 90e8600d78e5..123588c34dff 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use super::{DisplayAs, DisplayFormatType, ExecutionMode, PlanPropertiesCache}; +use super::{DisplayAs, DisplayFormatType, ExecutionMode, PlanProperties}; use crate::display::{display_orderings, ProjectSchemaDisplay}; use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; @@ -58,7 +58,7 @@ pub struct StreamingTableExec { projected_schema: SchemaRef, projected_output_ordering: Vec, infinite: bool, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl StreamingTableExec { @@ -87,7 +87,7 @@ impl StreamingTableExec { }; let projected_output_ordering = projected_output_ordering.into_iter().collect::>(); - let cache = Self::create_cache( + let cache = Self::compute_properties( projected_schema.clone(), &projected_output_ordering, &partitions, @@ -128,12 +128,12 @@ impl StreamingTableExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, orderings: &[LexOrdering], partitions: &[Arc], is_infinite: bool, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties: let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); @@ -147,7 +147,7 @@ impl StreamingTableExec { ExecutionMode::Bounded }; - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -195,7 +195,7 @@ impl ExecutionPlan for StreamingTableExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index a70e05809923..23df3753e817 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -27,7 +27,7 @@ use std::{ use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::{ common, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -121,7 +121,7 @@ pub struct MockExec { /// if true (the default), sends data using a separate task to to ensure the /// batches are not available without this stream yielding first use_task: bool, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl MockExec { @@ -133,7 +133,7 @@ impl MockExec { /// ensure any poll loops are correct. This behavior can be /// changed with `with_use_task` pub fn new(data: Vec>, schema: SchemaRef) -> Self { - let cache = Self::create_cache(schema.clone()); + let cache = Self::compute_properties(schema.clone()); Self { data, schema, @@ -151,10 +151,10 @@ impl MockExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -181,7 +181,7 @@ impl ExecutionPlan for MockExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -282,7 +282,7 @@ pub struct BarrierExec { /// all streams wait on this barrier to produce barrier: Arc, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl BarrierExec { @@ -290,7 +290,7 @@ impl BarrierExec { pub fn new(data: Vec>, schema: SchemaRef) -> Self { // wait for all streams and the input let barrier = Arc::new(Barrier::new(data.len() + 1)); - let cache = Self::create_cache(schema.clone(), &data); + let cache = Self::compute_properties(schema.clone(), &data); Self { data, schema, @@ -307,9 +307,12 @@ impl BarrierExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef, data: &[Vec]) -> PlanPropertiesCache { + fn compute_properties( + schema: SchemaRef, + data: &[Vec], + ) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(data.len()), ExecutionMode::Bounded, @@ -336,7 +339,7 @@ impl ExecutionPlan for BarrierExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -394,7 +397,7 @@ impl ExecutionPlan for BarrierExec { /// A mock execution plan that errors on a call to execute #[derive(Debug)] pub struct ErrorExec { - cache: PlanPropertiesCache, + cache: PlanProperties, } impl Default for ErrorExec { @@ -410,15 +413,15 @@ impl ErrorExec { DataType::Int64, true, )])); - let cache = Self::create_cache(schema.clone()); + let cache = Self::compute_properties(schema.clone()); Self { cache } } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -445,7 +448,7 @@ impl ExecutionPlan for ErrorExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -475,7 +478,7 @@ impl ExecutionPlan for ErrorExec { pub struct StatisticsExec { stats: Statistics, schema: Arc, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { @@ -484,7 +487,7 @@ impl StatisticsExec { .column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); - let cache = Self::create_cache(Arc::new(schema.clone())); + let cache = Self::compute_properties(Arc::new(schema.clone())); Self { stats, schema: Arc::new(schema), @@ -493,10 +496,10 @@ impl StatisticsExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(2), ExecutionMode::Bounded, @@ -528,7 +531,7 @@ impl ExecutionPlan for StatisticsExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -566,13 +569,13 @@ pub struct BlockingExec { /// Ref-counting helper to check if the plan and the produced stream are still in memory. refs: Arc<()>, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl BlockingExec { /// Create new [`BlockingExec`] with a give schema and number of partitions. pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { - let cache = Self::create_cache(schema.clone(), n_partitions); + let cache = Self::compute_properties(schema.clone(), n_partitions); Self { schema, refs: Default::default(), @@ -590,10 +593,10 @@ impl BlockingExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef, n_partitions: usize) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef, n_partitions: usize) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(n_partitions), ExecutionMode::Bounded, @@ -620,7 +623,7 @@ impl ExecutionPlan for BlockingExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -704,7 +707,7 @@ pub struct PanicExec { /// Number of output partitions. Each partition will produce this /// many empty output record batches prior to panicing batches_until_panics: Vec, - cache: PlanPropertiesCache, + cache: PlanProperties, } impl PanicExec { @@ -712,7 +715,7 @@ impl PanicExec { /// partitions, which will each panic immediately. pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { let batches_until_panics = vec![0; n_partitions]; - let cache = Self::create_cache(schema.clone(), &batches_until_panics); + let cache = Self::compute_properties(schema.clone(), &batches_until_panics); Self { schema, batches_until_panics, @@ -727,14 +730,14 @@ impl PanicExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, batches_until_panics: &[usize], - ) -> PlanPropertiesCache { + ) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); let num_partitions = batches_until_panics.len(); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(num_partitions), ExecutionMode::Bounded, @@ -761,7 +764,7 @@ impl ExecutionPlan for PanicExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 4d5377a9bdcc..a533f2249588 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -27,10 +27,10 @@ use std::task::{Context, Poll}; use std::{any::Any, sync::Arc}; use super::{ - exec_mode_flatten, + execution_mode_from_children, metrics::{ExecutionPlanMetricsSet, MetricsSet}, ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, Statistics, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::metrics::BaselineMetrics; use crate::stream::ObservedStream; @@ -91,14 +91,14 @@ pub struct UnionExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl UnionExec { /// Create a new UnionExec pub fn new(inputs: Vec>) -> Self { let schema = union_schema(&inputs); - let cache = Self::create_cache(&inputs, schema); + let cache = Self::compute_properties(&inputs, schema); UnionExec { inputs, metrics: ExecutionPlanMetricsSet::new(), @@ -112,10 +112,10 @@ impl UnionExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( inputs: &[Arc], schema: SchemaRef, - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties: // TODO: In some cases, we should be able to preserve some equivalence // classes and constants. Add support for such cases. @@ -161,9 +161,9 @@ impl UnionExec { let output_partitioning = Partitioning::UnknownPartitioning(num_partitions); // Determine execution mode: - let mode = exec_mode_flatten(inputs.iter()); + let mode = execution_mode_from_children(inputs.iter()); - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -187,7 +187,7 @@ impl ExecutionPlan for UnionExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } @@ -317,7 +317,7 @@ pub struct InterleaveExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl InterleaveExec { @@ -328,7 +328,7 @@ impl InterleaveExec { "Not all InterleaveExec children have a consistent hash partitioning" ); } - let cache = Self::create_cache(&inputs); + let cache = Self::compute_properties(&inputs); Ok(InterleaveExec { inputs, metrics: ExecutionPlanMetricsSet::new(), @@ -342,15 +342,15 @@ impl InterleaveExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(inputs: &[Arc]) -> PlanPropertiesCache { + fn compute_properties(inputs: &[Arc]) -> PlanProperties { let schema = union_schema(inputs); let eq_properties = EquivalenceProperties::new(schema); // Get output partitioning: let output_partitioning = inputs[0].output_partitioning().clone(); // Determine execution mode: - let mode = exec_mode_flatten(inputs.iter()); + let mode = execution_mode_from_children(inputs.iter()); - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -374,7 +374,7 @@ impl ExecutionPlan for InterleaveExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index d727091fd1c3..72fcbfd2ffb9 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -20,7 +20,7 @@ use std::{any::Any, sync::Arc}; use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use super::{DisplayAs, PlanPropertiesCache}; +use super::{DisplayAs, PlanProperties}; use crate::{ expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, @@ -60,7 +60,7 @@ pub struct UnnestExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl UnnestExec { @@ -71,7 +71,7 @@ impl UnnestExec { schema: SchemaRef, options: UnnestOptions, ) -> Self { - let cache = Self::create_cache(&input, schema.clone()); + let cache = Self::compute_properties(&input, schema.clone()); UnnestExec { input, schema, @@ -83,13 +83,13 @@ impl UnnestExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, schema: SchemaRef, - ) -> PlanPropertiesCache { + ) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, input.output_partitioning().clone(), input.execution_mode(), @@ -116,7 +116,7 @@ impl ExecutionPlan for UnnestExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index f31272879279..1e535b43d7c0 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::sync::Arc; use super::{ - common, DisplayAs, ExecutionMode, PlanPropertiesCache, SendableRecordBatchStream, + common, DisplayAs, ExecutionMode, PlanProperties, SendableRecordBatchStream, Statistics, }; use crate::{ @@ -43,7 +43,7 @@ pub struct ValuesExec { /// The data data: Vec, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl ValuesExec { @@ -114,7 +114,7 @@ impl ValuesExec { } } - let cache = Self::create_cache(schema.clone()); + let cache = Self::compute_properties(schema.clone()); Ok(ValuesExec { schema, data: batches, @@ -128,10 +128,10 @@ impl ValuesExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -159,7 +159,7 @@ impl ExecutionPlan for ValuesExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } 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 d7579cdc041d..eb4a27341785 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -35,7 +35,7 @@ use crate::windows::{ }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, - InputOrderMode, PlanPropertiesCache, RecordBatchStream, SendableRecordBatchStream, + InputOrderMode, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; @@ -90,7 +90,7 @@ pub struct BoundedWindowAggExec { // See `get_ordered_partition_by_indices` for more details. ordered_partition_by_indices: Vec, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl BoundedWindowAggExec { @@ -121,7 +121,7 @@ impl BoundedWindowAggExec { vec![] } }; - let cache = Self::create_cache(&input, &schema, &window_expr); + let cache = Self::compute_properties(&input, &schema, &window_expr); Ok(Self { input, window_expr, @@ -183,11 +183,11 @@ impl BoundedWindowAggExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( input: &Arc, schema: &SchemaRef, window_expr: &[Arc], - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties: let eq_properties = window_equivalence_properties(schema, input, window_expr); @@ -197,7 +197,7 @@ impl BoundedWindowAggExec { let output_partitioning = input.output_partitioning().clone(); // Construct properties cache - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, // Equivalence Properties output_partitioning, // Output Partitioning input.execution_mode(), // Execution Mode @@ -240,7 +240,7 @@ impl ExecutionPlan for BoundedWindowAggExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index f143d228f381..efc84526c9ad 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -31,7 +31,7 @@ use crate::windows::{ }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, - ExecutionPlan, PhysicalExpr, PlanPropertiesCache, RecordBatchStream, + ExecutionPlan, PhysicalExpr, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; @@ -65,7 +65,7 @@ pub struct WindowAggExec { // see `get_ordered_partition_by_indices` for more details. ordered_partition_by_indices: Vec, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl WindowAggExec { @@ -80,7 +80,7 @@ impl WindowAggExec { let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); - let cache = Self::create_cache(schema.clone(), &input, &window_expr); + let cache = Self::compute_properties(schema.clone(), &input, &window_expr); Ok(Self { input, window_expr, @@ -117,11 +117,11 @@ impl WindowAggExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache( + fn compute_properties( schema: SchemaRef, input: &Arc, window_expr: &[Arc], - ) -> PlanPropertiesCache { + ) -> PlanProperties { // Calculate equivalence properties: let eq_properties = window_equivalence_properties(&schema, input, window_expr); @@ -139,7 +139,7 @@ impl WindowAggExec { }; // Construct properties cache: - PlanPropertiesCache::new(eq_properties, output_partitioning, mode) + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -177,7 +177,7 @@ impl ExecutionPlan for WindowAggExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index 44a42a4fcf92..9acabf2447eb 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -25,9 +25,7 @@ use super::{ SendableRecordBatchStream, Statistics, }; use crate::memory::MemoryStream; -use crate::{ - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, PlanPropertiesCache, -}; +use crate::{DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, PlanProperties}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -85,13 +83,13 @@ pub struct WorkTableExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanPropertiesCache, + cache: PlanProperties, } impl WorkTableExec { /// Create a new execution plan for a worktable exec. pub fn new(name: String, schema: SchemaRef) -> Self { - let cache = Self::create_cache(schema.clone()); + let cache = Self::compute_properties(schema.clone()); Self { name, schema, @@ -112,10 +110,10 @@ impl WorkTableExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn create_cache(schema: SchemaRef) -> PlanPropertiesCache { + fn compute_properties(schema: SchemaRef) -> PlanProperties { let eq_properties = EquivalenceProperties::new(schema); - PlanPropertiesCache::new( + PlanProperties::new( eq_properties, Partitioning::UnknownPartitioning(1), ExecutionMode::Bounded, @@ -142,7 +140,7 @@ impl ExecutionPlan for WorkTableExec { self } - fn cache(&self) -> &PlanPropertiesCache { + fn properties(&self) -> &PlanProperties { &self.cache } From a0128449970fa4cefa659b110ea5de8123ed5f8b Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Tue, 27 Feb 2024 19:09:54 +0300 Subject: [PATCH 09/15] Update datafusion/physical-plan/src/lib.rs Co-authored-by: Andrew Lamb --- datafusion/physical-plan/src/lib.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index d18b8f238859..cebe3a8837ad 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -460,6 +460,9 @@ pub enum ExecutionMode { /// Represents the mode where generated stream is unbounded, e.g. infinite. /// Even though the operator generates an unbounded stream of results, it /// works with bounded memory and execution can still continue successfully. + /// + /// The stream that results from calling `execute` on an `ExecutionPlan` that is `Unbounded` + /// will never be done (return `None`), except in case of error. Unbounded, /// Represents the mode where some of the operator's input stream(s) are /// unbounded; however, the operator cannot generate streaming results from From e4a994772b17d0e3f30e467ce1a685533062c7e7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 27 Feb 2024 19:13:08 +0300 Subject: [PATCH 10/15] Update comments --- datafusion/physical-plan/src/lib.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index cebe3a8837ad..e21a96cee2ff 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -461,7 +461,7 @@ pub enum ExecutionMode { /// Even though the operator generates an unbounded stream of results, it /// works with bounded memory and execution can still continue successfully. /// - /// The stream that results from calling `execute` on an `ExecutionPlan` that is `Unbounded` + /// The stream that results from calling `execute` on an `ExecutionPlan` that is `Unbounded` /// will never be done (return `None`), except in case of error. Unbounded, /// Represents the mode where some of the operator's input stream(s) are @@ -511,8 +511,8 @@ fn execution_mode_from_children<'a>( /// Stores the plan properties used in query optimization. /// -/// This struct holds various properties useful for the query planning, which are used -/// during optimization and execution phases. +/// These properties are in a single structure to permit this information to be computed +/// once and then those cached results used multiple times without recomputation (aka a cache) #[derive(Debug, Clone)] pub struct PlanProperties { /// Stores the [`EquivalenceProperties`] of the [`ExecutionPlan`]. From a940a46a6677c6f9639272f12164ba8a71b2b7c4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 27 Feb 2024 19:44:12 +0300 Subject: [PATCH 11/15] Move properties to another trait. --- datafusion/core/src/dataframe/mod.rs | 2 +- .../core/src/datasource/listing/table.rs | 1 + datafusion/core/src/datasource/memory.rs | 5 +- .../datasource/physical_plan/arrow_file.rs | 2 +- .../core/src/datasource/physical_plan/csv.rs | 3 +- .../core/src/datasource/physical_plan/json.rs | 3 +- .../datasource/physical_plan/parquet/mod.rs | 12 +- .../enforce_distribution.rs | 2 + .../src/physical_optimizer/enforce_sorting.rs | 5 +- .../src/physical_optimizer/join_selection.rs | 2 + .../limited_distinct_aggregation.rs | 1 + .../physical_optimizer/output_requirements.rs | 4 +- .../physical_optimizer/pipeline_checker.rs | 1 + .../physical_optimizer/projection_pushdown.rs | 1 + .../replace_with_order_preserving_variants.rs | 1 + .../src/physical_optimizer/sort_pushdown.rs | 3 +- .../physical_optimizer/topk_aggregation.rs | 3 +- .../core/src/physical_optimizer/utils.rs | 1 + datafusion/core/src/physical_planner.rs | 1 + .../physical-plan/src/aggregates/mod.rs | 4 +- .../physical-plan/src/aggregates/row_hash.rs | 3 +- datafusion/physical-plan/src/analyze.rs | 5 +- .../physical-plan/src/coalesce_batches.rs | 2 +- .../physical-plan/src/coalesce_partitions.rs | 5 +- datafusion/physical-plan/src/common.rs | 2 +- datafusion/physical-plan/src/filter.rs | 4 +- datafusion/physical-plan/src/insert.rs | 4 +- .../physical-plan/src/joins/cross_join.rs | 1 + .../physical-plan/src/joins/hash_join.rs | 1 + .../src/joins/nested_loop_join.rs | 4 +- .../src/joins/sort_merge_join.rs | 12 +- .../src/joins/symmetric_hash_join.rs | 4 +- datafusion/physical-plan/src/lib.rs | 134 +++++++++++------- datafusion/physical-plan/src/limit.rs | 4 +- datafusion/physical-plan/src/projection.rs | 3 +- .../physical-plan/src/repartition/mod.rs | 4 +- .../physical-plan/src/sorts/partial_sort.rs | 4 +- datafusion/physical-plan/src/sorts/sort.rs | 3 +- .../src/sorts/sort_preserving_merge.rs | 4 +- datafusion/physical-plan/src/union.rs | 7 +- datafusion/physical-plan/src/unnest.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 4 +- datafusion/physical-plan/src/windows/mod.rs | 2 +- .../src/windows/window_agg_exec.rs | 4 +- 44 files changed, 170 insertions(+), 109 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 7898b71e23f6..1372570179fe 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1517,7 +1517,7 @@ mod tests { WindowFunctionDefinition, }; use datafusion_physical_expr::expressions::Column; - use datafusion_physical_plan::get_plan_string; + use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties}; // Get string representation of the plan async fn assert_physical_plan(df: &DataFrame, expected: Vec<&str>) { diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 56e64f556c12..a1f3d14aacca 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -920,6 +920,7 @@ mod tests { use datafusion_common::{assert_contains, GetExt, ScalarValue}; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use datafusion_physical_expr::PhysicalSortExpr; + use datafusion_physical_plan::ExecutionPlanProperties; use tempfile::TempDir; #[tokio::test] diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 901e74dfc218..b4a51be264eb 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -32,6 +32,7 @@ use datafusion_common::{ not_impl_err, plan_err, Constraints, DFSchema, DataFusionError, SchemaExt, }; use datafusion_execution::TaskContext; +use datafusion_physical_plan::ExecutionPlanProperties; use parking_lot::Mutex; use tokio::sync::RwLock; use tokio::task::JoinSet; @@ -161,10 +162,10 @@ impl MemTable { let exec = MemoryExec::try_new(&data, schema.clone(), None)?; if let Some(num_partitions) = output_partitions { - let exec = RepartitionExec::try_new( + let exec = Arc::new(RepartitionExec::try_new( Arc::new(exec), Partitioning::RoundRobinBatch(num_partitions), - )?; + )?) as Arc; // execute and collect results let mut output_partitions = vec![]; diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 8f010f1dcbf8..817f330097ac 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -152,7 +152,7 @@ impl ExecutionPlan for ArrowExec { 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.output_ordering().is_some()) + .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 { diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index a509121a82c8..3066d77acf45 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -43,6 +43,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; +use datafusion_physical_plan::ExecutionPlanProperties; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; @@ -197,7 +198,7 @@ impl ExecutionPlan for CsvExec { let repartitioned_file_groups_option = FileGroupPartitioner::new() .with_target_partitions(target_partitions) - .with_preserve_order_within_groups(self.output_ordering().is_some()) + .with_preserve_order_within_groups(self.cache.output_ordering().is_some()) .with_repartition_file_min_size(repartition_file_min_size) .repartition_file_groups(&self.base_config.file_groups); diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 7b0e84c4410b..c471035bd286 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -42,6 +42,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; +use datafusion_physical_plan::ExecutionPlanProperties; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{self, GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; @@ -150,7 +151,7 @@ impl ExecutionPlan for NdJsonExec { config: &datafusion_common::config::ConfigOptions, ) -> Result>> { let repartition_file_min_size = config.optimizer.repartition_file_min_size; - let preserve_order_within_groups = self.output_ordering().is_some(); + let preserve_order_within_groups = self.cache.output_ordering().is_some(); let file_groups = &self.base_config.file_groups; let repartitioned_file_groups_option = FileGroupPartitioner::new() diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 997bdae762b8..e6962040e8ac 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -48,6 +48,7 @@ use arrow::error::ArrowError; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; use bytes::Bytes; +use datafusion_physical_plan::ExecutionPlanProperties; use futures::future::BoxFuture; use futures::{StreamExt, TryStreamExt}; use itertools::Itertools; @@ -362,7 +363,7 @@ impl ExecutionPlan for ParquetExec { 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.output_ordering().is_some()) + .with_preserve_order_within_groups(self.cache.output_ordering().is_some()) .repartition_file_groups(&self.base_config.file_groups); let mut new_plan = self.clone(); @@ -1557,7 +1558,7 @@ mod tests { expected_row_num: Option, file_schema: SchemaRef, ) -> Result<()> { - let parquet_exec = ParquetExec::new( + let parquet_exec = Arc::new(ParquetExec::new( FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups, @@ -1570,7 +1571,7 @@ mod tests { }, None, None, - ); + )) as Arc; assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); let results = parquet_exec.execute(0, state.task_ctx())?.next().await; @@ -1685,7 +1686,10 @@ mod tests { None, None, ); - assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); + assert_eq!( + parquet_exec.cache.output_partitioning().partition_count(), + 1 + ); assert_eq!(parquet_exec.schema().as_ref(), &expected_schema); let mut results = parquet_exec.execute(0, task_ctx)?; diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index c7ffc7838b36..c08aadc33c74 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -56,6 +56,7 @@ use datafusion_physical_expr::{ use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; +use datafusion_physical_plan::ExecutionPlanProperties; use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -1404,6 +1405,7 @@ pub(crate) mod tests { // model that it requires the output ordering of its input fn required_input_ordering(&self) -> Vec>> { vec![self + .cache .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs)] } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index b459c86518b6..25280261c0a0 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -65,6 +65,7 @@ use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; +use datafusion_physical_plan::ExecutionPlanProperties; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -390,7 +391,7 @@ fn analyze_immediate_sort_removal( // If this sort is unnecessary, we should remove it: if sort_input .equivalence_properties() - .ordering_satisfy(sort_exec.output_ordering().unwrap_or(&[])) + .ordering_satisfy(node.plan.output_ordering().unwrap_or(&[])) { node.plan = if !sort_exec.preserve_partitioning() && sort_input.output_partitioning().partition_count() > 1 @@ -573,7 +574,7 @@ fn remove_corresponding_sort_from_sub_plan( { node.plan = Arc::new(RepartitionExec::try_new( node.children[0].plan.clone(), - repartition.output_partitioning().clone(), + repartition.properties().output_partitioning().clone(), )?) as _; } }; diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index f74732305372..349e33dae251 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -42,6 +42,7 @@ use datafusion_common::{internal_err, DataFusionError, JoinSide, JoinType}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::sort_properties::SortProperties; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_plan::ExecutionPlanProperties; /// The [`JoinSelection`] rule tries to modify a given plan so that it can /// accommodate infinite sources and optimize joins in the plan according to @@ -1377,6 +1378,7 @@ mod hash_join_tests { use arrow::record_batch::RecordBatch; use datafusion_common::utils::DataPtr; use datafusion_common::JoinType; + use datafusion_physical_plan::ExecutionPlanProperties; use std::sync::Arc; struct TestCase { diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index 9855247151b8..036c938c1ca6 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -25,6 +25,7 @@ use crate::physical_plan::ExecutionPlan; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; +use datafusion_physical_plan::ExecutionPlanProperties; use itertools::Itertools; use std::sync::Arc; diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 992a6e7f82c0..da0697eb9aba 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -33,7 +33,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion_physical_plan::PlanProperties; +use datafusion_physical_plan::{ExecutionPlanProperties, PlanProperties}; /// This rule either adds or removes [`OutputRequirements`]s to/from the physical /// plan according to its `mode` attribute, which is set by the constructors @@ -242,7 +242,7 @@ fn require_top_ordering_helper( if children.len() != 1 { Ok((plan, false)) } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let req_ordering = sort_exec.output_ordering().unwrap_or(&[]); + let req_ordering = plan.output_ordering().unwrap_or(&[]); let req_dist = sort_exec.required_input_distribution()[0].clone(); let reqs = PhysicalSortRequirement::from_sort_exprs(req_ordering); Ok(( diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 9a7afcb6409d..dcf9ef6f48b7 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -31,6 +31,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; +use datafusion_physical_plan::ExecutionPlanProperties; /// The PipelineChecker rule rejects non-runnable query plans that use /// pipeline-breaking operators on infinite input(s). diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 79d22374f9c2..1b9bf17bffa4 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -53,6 +53,7 @@ use datafusion_physical_expr::{ use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; +use datafusion_physical_plan::ExecutionPlanProperties; use itertools::Itertools; /// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to 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 8825feb45e98..be4b7d13bc7e 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 @@ -32,6 +32,7 @@ use datafusion_common::tree_node::Transformed; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::tree_node::PlanContext; +use datafusion_physical_plan::ExecutionPlanProperties; use itertools::izip; /// For a given `plan`, this object carries the information one needs from its diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 0efa908cf5fc..a5b1f12147f3 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -37,6 +37,7 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_plan::ExecutionPlanProperties; /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total @@ -262,7 +263,7 @@ fn try_pushdown_requirements_to_join( &smj.maintains_input_order(), Some(probe_side), ); - let mut smj_eqs = smj.equivalence_properties().clone(); + let mut smj_eqs = smj.properties().equivalence_properties().clone(); // smj will have this ordering when its input changes. smj_eqs = smj_eqs.with_reorder(new_output_ordering.unwrap_or_default()); let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index dd0261420304..7459deb1f72a 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -30,6 +30,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_plan::ExecutionPlanProperties; use itertools::Itertools; use std::sync::Arc; @@ -86,7 +87,7 @@ impl TopKAggregation { let children = sort.children(); let child = children.iter().exactly_one().ok()?; - let order = sort.output_ordering()?; + let order = plan.output_ordering()?; let order = order.iter().exactly_one().ok()?; let limit = sort.fetch()?; diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 4f4b17345ef8..f4c2c3873f68 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -30,6 +30,7 @@ use crate::physical_plan::ExecutionPlan; use datafusion_physical_expr::{LexRequirement, PhysicalSortRequirement}; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::tree_node::PlanContext; +use datafusion_physical_plan::ExecutionPlanProperties; /// This utility function adds a `SortExec` above an operator according to the /// given ordering requirements while preserving the original partitioning. diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index debde6aa4571..b16cc77a43f2 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -96,6 +96,7 @@ use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_sql::utils::window_expr_common_partition_keys; use async_trait::async_trait; +use datafusion_physical_plan::ExecutionPlanProperties; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::{multiunzip, Itertools}; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 98d41cca6764..f96eacf45896 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use super::{DisplayAs, ExecutionMode, PlanProperties}; +use super::{DisplayAs, ExecutionMode, ExecutionPlanProperties, PlanProperties}; use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, @@ -496,7 +496,7 @@ impl AggregateExec { return false; } // ensure there is no output ordering; can this rule be relaxed? - if self.output_ordering().is_some() { + if self.cache.output_ordering().is_some() { return false; } // ensure no ordering is required on the input diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index f9db0a050cfc..8036012b7262 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -32,7 +32,7 @@ use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::sorts::sort::{read_spill_as_stream, sort_batch}; use crate::sorts::streaming_merge; use crate::stream::RecordBatchStreamAdapter; -use crate::{aggregates, ExecutionPlan, PhysicalExpr}; +use crate::{aggregates, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; @@ -341,6 +341,7 @@ impl GroupedHashAggregateStream { .with_can_spill(true) .register(context.memory_pool()); let (ordering, _) = agg + .cache .equivalence_properties() .find_longest_permutation(&agg_group_by.output_exprs()); let group_ordering = GroupOrdering::try_new( diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index fed4b97d2afb..f771ac238887 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -21,7 +21,10 @@ use std::sync::Arc; use std::{any::Any, time::Instant}; use super::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; -use super::{DisplayAs, Distribution, PlanProperties, SendableRecordBatchStream}; +use super::{ + DisplayAs, Distribution, ExecutionPlanProperties, PlanProperties, + SendableRecordBatchStream, +}; use crate::display::DisplayableExecutionPlan; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 055f16288f95..0b9ecebbb1e8 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::{DisplayAs, PlanProperties, Statistics}; +use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, Statistics}; use crate::{ DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 7037445164a3..3c5b7e9c13fb 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -23,7 +23,10 @@ use std::sync::Arc; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::stream::{ObservedStream, RecordBatchReceiverStream}; -use super::{DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; +use super::{ + DisplayAs, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, + Statistics, +}; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 5172bc9b2a3c..003c60edd9a8 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -22,7 +22,7 @@ use std::fs::{metadata, File}; use std::path::{Path, PathBuf}; use std::sync::Arc; -use super::SendableRecordBatchStream; +use super::{ExecutionPlanProperties, SendableRecordBatchStream}; use crate::stream::RecordBatchReceiverStream; use crate::{ColumnStatistics, ExecutionPlan, Statistics}; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 95c09f541cc2..4155b00820f4 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -24,8 +24,8 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::{ - ColumnStatistics, DisplayAs, PlanProperties, RecordBatchStream, - SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index fd0bec108e03..7d44828ac243 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -23,8 +23,8 @@ use std::fmt::Debug; use std::sync::Arc; use super::{ - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, Partitioning, + PlanProperties, SendableRecordBatchStream, }; use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index f73f3d36cdb4..9f8dc0ce56b0 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -26,6 +26,7 @@ use super::utils::{ use crate::coalesce_batches::concat_batches; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::ExecutionPlanProperties; use crate::{ execution_mode_from_children, ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, PlanProperties, RecordBatchStream, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 2fdb2a17ebe8..ee3438c6a363 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -27,6 +27,7 @@ use super::{ utils::{OnceAsync, OnceFut}, PartitionMode, }; +use crate::ExecutionPlanProperties; use crate::{ coalesce_partitions::CoalescePartitionsExec, execution_mode_from_children, handle_state, diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 5d2175d4a820..6fe28c8b54f3 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -35,8 +35,8 @@ use crate::joins::utils::{ use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ execution_mode_from_children, DisplayAs, DisplayFormatType, Distribution, - ExecutionMode, ExecutionPlan, PlanProperties, RecordBatchStream, - SendableRecordBatchStream, + ExecutionMode, ExecutionPlan, ExecutionPlanProperties, PlanProperties, + RecordBatchStream, SendableRecordBatchStream, }; use arrow::array::{ diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index bde831b731ba..7b70a2952b4c 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -38,8 +38,8 @@ use crate::joins::utils::{ use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ execution_mode_from_children, metrics, DisplayAs, DisplayFormatType, Distribution, - ExecutionPlan, PhysicalExpr, PlanProperties, RecordBatchStream, - SendableRecordBatchStream, Statistics, + ExecutionPlan, ExecutionPlanProperties, PhysicalExpr, PlanProperties, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::*; @@ -189,16 +189,16 @@ impl SortMergeJoinExec { &self.on } - pub fn right(&self) -> &dyn ExecutionPlan { - self.right.as_ref() + pub fn right(&self) -> &Arc { + &self.right } pub fn join_type(&self) -> JoinType { self.join_type } - pub fn left(&self) -> &dyn ExecutionPlan { - self.left.as_ref() + pub fn left(&self) -> &Arc { + &self.left } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 77871a8b5483..6460b2b478ef 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -50,8 +50,8 @@ use crate::{ expressions::PhysicalSortExpr, joins::StreamJoinPartitionMode, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, PlanProperties, - RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::{ diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index e21a96cee2ff..e37f84fb0150 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -127,33 +127,6 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { fn properties(&self) -> &PlanProperties; - /// Specifies how the output of this `ExecutionPlan` is split into - /// partitions. - fn output_partitioning(&self) -> &Partitioning { - &self.properties().partitioning - } - - /// Specifies whether this plan generates an infinite stream of records. - /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. - fn execution_mode(&self) -> ExecutionMode { - self.properties().exec_mode - } - - /// If the output of this `ExecutionPlan` within each partition is sorted, - /// returns `Some(keys)` with the description of how it was sorted. - /// - /// For example, Sort, (obviously) produces sorted output as does - /// SortPreservingMergeStream. Less obviously `Projection` - /// produces sorted output if its input was sorted as it does not - /// reorder the input rows, - /// - /// It is safe to return `None` here if your `ExecutionPlan` does not - /// have any particular output order here - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.properties().output_ordering.as_deref() - } - /// Specifies the data distribution requirements for all the /// children for this `ExecutionPlan`, By default it's [[Distribution::UnspecifiedDistribution]] for each child, fn required_input_distribution(&self) -> Vec { @@ -212,27 +185,6 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { .collect() } - /// Get the [`EquivalenceProperties`] within the plan. - /// - /// Equivalence properties tell DataFusion what columns are known to be - /// equal, during various optimization passes. By default, this returns "no - /// known equivalences" which is always correct, but may cause DataFusion to - /// unnecessarily resort data. - /// - /// If this ExecutionPlan makes no changes to the schema of the rows flowing - /// through it or how columns within each row relate to each other, it - /// should return the equivalence properties of its input. For - /// example, since `FilterExec` may remove rows from its input, but does not - /// otherwise modify them, it preserves its input equivalence properties. - /// However, since `ProjectionExec` may calculate derived expressions, it - /// needs special handling. - /// - /// See also [`Self::maintains_input_order`] and [`Self::output_ordering`] - /// for related concepts. - fn equivalence_properties(&self) -> &EquivalenceProperties { - &self.properties().eq_properties - } - /// Get a list of children `ExecutionPlan`s that act as inputs to this plan. /// The returned list will be empty for leaf nodes such as scans, will contain /// a single value for unary nodes, or two values for binary nodes (such as @@ -450,6 +402,66 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } } +pub trait ExecutionPlanProperties { + fn output_partitioning(&self) -> &Partitioning; + + fn execution_mode(&self) -> ExecutionMode; + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]>; + + fn equivalence_properties(&self) -> &EquivalenceProperties; +} + +impl ExecutionPlanProperties for Arc { + /// Specifies how the output of this `ExecutionPlan` is split into + /// partitions. + fn output_partitioning(&self) -> &Partitioning { + self.properties().output_partitioning() + } + + /// Specifies whether this plan generates an infinite stream of records. + /// If the plan does not support pipelining, but its input(s) are + /// infinite, returns an error to indicate this. + fn execution_mode(&self) -> ExecutionMode { + self.properties().execution_mode() + } + + /// If the output of this `ExecutionPlan` within each partition is sorted, + /// returns `Some(keys)` with the description of how it was sorted. + /// + /// For example, Sort, (obviously) produces sorted output as does + /// SortPreservingMergeStream. Less obviously `Projection` + /// produces sorted output if its input was sorted as it does not + /// reorder the input rows, + /// + /// It is safe to return `None` here if your `ExecutionPlan` does not + /// have any particular output order here + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + self.properties().output_ordering() + } + + /// Get the [`EquivalenceProperties`] within the plan. + /// + /// Equivalence properties tell DataFusion what columns are known to be + /// equal, during various optimization passes. By default, this returns "no + /// known equivalences" which is always correct, but may cause DataFusion to + /// unnecessarily resort data. + /// + /// If this ExecutionPlan makes no changes to the schema of the rows flowing + /// through it or how columns within each row relate to each other, it + /// should return the equivalence properties of its input. For + /// example, since `FilterExec` may remove rows from its input, but does not + /// otherwise modify them, it preserves its input equivalence properties. + /// However, since `ProjectionExec` may calculate derived expressions, it + /// needs special handling. + /// + /// See also [`Self::maintains_input_order`] and [`Self::output_ordering`] + /// for related concepts. + fn equivalence_properties(&self) -> &EquivalenceProperties { + self.properties().equivalence_properties() + } +} + /// Describes the execution mode of an operator's resulting stream with respect /// to its size and behavior. There are three possible execution modes: `Bounded`, /// `Unbounded` and `PipelineBreaking`. @@ -564,6 +576,22 @@ impl PlanProperties { self } + pub fn equivalence_properties(&self) -> &EquivalenceProperties { + &self.eq_properties + } + + pub fn output_partitioning(&self) -> &Partitioning { + &self.partitioning + } + + pub fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + self.output_ordering.as_deref() + } + + pub fn execution_mode(&self) -> ExecutionMode { + self.exec_mode + } + /// Get schema of the node. fn schema(&self) -> &SchemaRef { self.eq_properties.schema() @@ -577,11 +605,8 @@ impl PlanProperties { /// 2. CoalescePartitionsExec for collapsing all of the partitions into one without ordering guarantee /// 3. SortPreservingMergeExec for collapsing all of the sorted partitions into one with ordering guarantee pub fn need_data_exchange(plan: Arc) -> bool { - if let Some(repart) = plan.as_any().downcast_ref::() { - !matches!( - repart.output_partitioning(), - Partitioning::RoundRobinBatch(_) - ) + if let Some(_) = plan.as_any().downcast_ref::() { + !matches!(plan.output_partitioning(), Partitioning::RoundRobinBatch(_)) } else if let Some(coalesce) = plan.as_any().downcast_ref::() { coalesce.input().output_partitioning().partition_count() > 1 @@ -652,7 +677,8 @@ pub fn execute_stream( 1 => plan.execute(0, context), _ => { // merge into a single partition - let plan = CoalescePartitionsExec::new(plan.clone()); + let plan = Arc::new(CoalescePartitionsExec::new(plan.clone())) + as Arc; // CoalescePartitionsExec must produce a single partition assert_eq!(1, plan.output_partitioning().partition_count()); plan.execute(0, context) diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 1c09ec88ae08..3520d45f9e6c 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -24,7 +24,7 @@ use std::task::{Context, Poll}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{ - DisplayAs, ExecutionMode, PlanProperties, RecordBatchStream, + DisplayAs, ExecutionMode, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::{DisplayFormatType, Distribution, ExecutionPlan, Partitioning}; @@ -410,7 +410,7 @@ impl ExecutionPlan for LocalLimitExec { _ => Statistics { // the result output row number will always be no greater than the limit number num_rows: Precision::Inexact( - self.fetch * self.output_partitioning().partition_count(), + self.fetch * self.cache.output_partitioning().partition_count(), ), column_statistics: col_stats, diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 402feeaf80ba..8fe82e7de3eb 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,7 +29,8 @@ use std::task::{Context, Poll}; use super::expressions::Column; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{ - DisplayAs, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; use crate::{ ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 1cbc71471eb2..d1befb7c53c0 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -26,7 +26,9 @@ use std::{any::Any, vec}; use super::common::SharedMemoryReservation; use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream}; +use super::{ + DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream, +}; use crate::common::{transpose, SpawnedTask}; use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 01e592d29f7b..500df6153fdb 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -61,8 +61,8 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::sort::sort_batch; use crate::{ - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PlanProperties, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::compute::concat_batches; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a96834cf0553..f46958663252 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -37,7 +37,8 @@ use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::topk::TopK; use crate::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionMode, - ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, + ExecutionPlan, ExecutionPlanProperties, Partitioning, PlanProperties, + SendableRecordBatchStream, Statistics, }; use arrow::compute::{concat_batches, lexsort_to_indices, take}; diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 862146e10549..e1e197e205e2 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -25,8 +25,8 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::streaming_merge; use crate::{ - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PlanProperties, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; use datafusion_common::{internal_err, DataFusionError, Result}; diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index a533f2249588..9f637361ff8f 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -29,8 +29,9 @@ use std::{any::Any, sync::Arc}; use super::{ execution_mode_from_children, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, + ExecutionPlanProperties, Partitioning, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; use crate::metrics::BaselineMetrics; use crate::stream::ObservedStream; @@ -204,7 +205,7 @@ impl ExecutionPlan for UnionExec { // which is the "meet" of all input orderings. In this example, this // function will return vec![false, true, true], indicating that we // preserve the orderings for the 2nd and the 3rd children. - if let Some(output_ordering) = self.output_ordering() { + if let Some(output_ordering) = self.cache.output_ordering() { self.inputs() .iter() .map(|child| { diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 72fcbfd2ffb9..776d98f5b6ea 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -20,7 +20,7 @@ use std::{any::Any, sync::Arc}; use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use super::{DisplayAs, PlanProperties}; +use super::{DisplayAs, ExecutionPlanProperties, PlanProperties}; use crate::{ expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, 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 eb4a27341785..c99ec599596d 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -35,8 +35,8 @@ use crate::windows::{ }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, - InputOrderMode, PlanProperties, RecordBatchStream, SendableRecordBatchStream, - Statistics, WindowExpr, + ExecutionPlanProperties, InputOrderMode, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, Statistics, WindowExpr, }; use arrow::{ diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0e20c88619c2..c19694aef8b7 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -27,7 +27,7 @@ use crate::{ cume_dist, dense_rank, lag, lead, percent_rank, rank, Literal, NthValue, Ntile, PhysicalSortExpr, RowNumber, }, - udaf, ExecutionPlan, InputOrderMode, PhysicalExpr, + udaf, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, PhysicalExpr, }; use arrow::datatypes::Schema; diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index efc84526c9ad..66d31ec94d1a 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -31,8 +31,8 @@ use crate::windows::{ }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionMode, - ExecutionPlan, PhysicalExpr, PlanProperties, RecordBatchStream, - SendableRecordBatchStream, Statistics, WindowExpr, + ExecutionPlan, ExecutionPlanProperties, PhysicalExpr, PlanProperties, + RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; use arrow::array::ArrayRef; From 77e5c35e9164ad99683ac41c04089145ae46a482 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 27 Feb 2024 15:57:44 -0800 Subject: [PATCH 12/15] Final review --- .../src/physical_optimizer/enforce_distribution.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index c7ffc7838b36..a6d5aa763253 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -443,10 +443,10 @@ where join_plan.plan = join_constructor((new_join_on, new_sort_options))?; } } - let mut requirements = join_plan; - requirements.children[0].data = left_keys; - requirements.children[1].data = right_keys; - Ok(requirements) + + join_plan.children[0].data = left_keys; + join_plan.children[1].data = right_keys; + Ok(join_plan) } fn reorder_aggregate_keys( @@ -1299,8 +1299,7 @@ pub(crate) mod tests { 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::ParquetExec; - use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; + use crate::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_optimizer::test_utils::{ From 930ac87d547f38b07be7cc4f1b2b5f934ca4e9c5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 28 Feb 2024 09:33:39 +0300 Subject: [PATCH 13/15] Resolve linter errors --- datafusion-cli/src/exec.rs | 2 +- datafusion/core/benches/sort.rs | 1 + datafusion/core/src/datasource/memory.rs | 6 ++--- .../datasource/physical_plan/arrow_file.rs | 4 +++- .../core/src/datasource/physical_plan/avro.rs | 24 ++++++++++++++++--- .../core/src/datasource/physical_plan/csv.rs | 6 +++-- .../core/src/datasource/physical_plan/json.rs | 4 ++-- .../datasource/physical_plan/parquet/mod.rs | 18 ++++++++++---- .../enforce_distribution.rs | 4 ++-- .../src/physical_optimizer/enforce_sorting.rs | 4 ++-- .../physical_optimizer/output_requirements.rs | 2 +- .../physical_optimizer/projection_pushdown.rs | 2 +- .../replace_with_order_preserving_variants.rs | 2 +- .../physical_optimizer/topk_aggregation.rs | 3 +-- datafusion/core/src/physical_planner.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 2 +- .../physical-plan/src/aggregates/row_hash.rs | 4 ++-- .../physical-plan/src/coalesce_partitions.rs | 5 +++- datafusion/physical-plan/src/display.rs | 4 ---- .../physical-plan/src/joins/test_utils.rs | 2 +- datafusion/physical-plan/src/lib.rs | 18 +++++++------- datafusion/physical-plan/src/limit.rs | 3 ++- datafusion/physical-plan/src/memory.rs | 9 ++++--- datafusion/physical-plan/src/stream.rs | 2 +- datafusion/physical-plan/src/union.rs | 12 +++++++--- 25 files changed, 92 insertions(+), 53 deletions(-) diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index 7e46f4a513fb..59581e91e857 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -36,7 +36,7 @@ use datafusion::datasource::listing::ListingTableUrl; use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::dml::CopyTo; use datafusion::logical_expr::{CreateExternalTable, DdlStatement, LogicalPlan}; -use datafusion::physical_plan::{collect, execute_stream}; +use datafusion::physical_plan::{collect, execute_stream, ExecutionPlanProperties}; use datafusion::prelude::SessionContext; use datafusion::sql::parser::{DFParser, Statement}; use datafusion::sql::sqlparser::dialect::dialect_from_str; diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index fbb94d66db58..34b4a5ebf0dc 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -95,6 +95,7 @@ use rand::{Rng, SeedableRng}; use tokio::runtime::Runtime; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::ExecutionPlanProperties; /// Total number of streams to divide each input into /// models 8 partition plan (should it be 16??) diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index b4a51be264eb..1ac492a76040 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -162,14 +162,14 @@ impl MemTable { let exec = MemoryExec::try_new(&data, schema.clone(), None)?; if let Some(num_partitions) = output_partitions { - let exec = Arc::new(RepartitionExec::try_new( + let exec = RepartitionExec::try_new( Arc::new(exec), Partitioning::RoundRobinBatch(num_partitions), - )?) as Arc; + )?; // execute and collect results let mut output_partitions = vec![]; - for i in 0..exec.output_partitioning().partition_count() { + for i in 0..exec.properties().output_partitioning().partition_count() { // execute this *output* partition and collect all batches let task_ctx = state.task_ctx(); let mut stream = exec.execute(i, task_ctx)?; diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 817f330097ac..82774a6e831c 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -152,7 +152,9 @@ impl ExecutionPlan for ArrowExec { 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( + self.properties().output_ordering().is_some(), + ) .repartition_file_groups(&self.base_config.file_groups); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 2b913d862576..6e7dcf39069c 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -279,7 +279,13 @@ mod tests { table_partition_cols: vec![], output_ordering: vec![], }); - assert_eq!(avro_exec.output_partitioning().partition_count(), 1); + assert_eq!( + avro_exec + .properties() + .output_partitioning() + .partition_count(), + 1 + ); let mut results = avro_exec .execute(0, state.task_ctx()) .expect("plan execution failed"); @@ -350,7 +356,13 @@ mod tests { table_partition_cols: vec![], output_ordering: vec![], }); - assert_eq!(avro_exec.output_partitioning().partition_count(), 1); + assert_eq!( + avro_exec + .properties() + .output_partitioning() + .partition_count(), + 1 + ); let mut results = avro_exec .execute(0, state.task_ctx()) @@ -420,7 +432,13 @@ mod tests { table_partition_cols: vec![Field::new("date", DataType::Utf8, false)], output_ordering: vec![], }); - assert_eq!(avro_exec.output_partitioning().partition_count(), 1); + assert_eq!( + avro_exec + .properties() + .output_partitioning() + .partition_count(), + 1 + ); let mut results = avro_exec .execute(0, state.task_ctx()) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 3066d77acf45..05a83e8ac0b7 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -41,9 +41,9 @@ use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_plan::ExecutionPlanProperties; use bytes::{Buf, Bytes}; -use datafusion_physical_plan::ExecutionPlanProperties; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; @@ -198,7 +198,9 @@ impl ExecutionPlan for CsvExec { let repartitioned_file_groups_option = FileGroupPartitioner::new() .with_target_partitions(target_partitions) - .with_preserve_order_within_groups(self.cache.output_ordering().is_some()) + .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); diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index c471035bd286..6f9af2e6abcf 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -40,9 +40,9 @@ use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_plan::ExecutionPlanProperties; use bytes::{Buf, Bytes}; -use datafusion_physical_plan::ExecutionPlanProperties; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{self, GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; @@ -151,7 +151,7 @@ impl ExecutionPlan for NdJsonExec { config: &datafusion_common::config::ConfigOptions, ) -> Result>> { let repartition_file_min_size = config.optimizer.repartition_file_min_size; - let preserve_order_within_groups = self.cache.output_ordering().is_some(); + 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() diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index e6962040e8ac..5ccffde26359 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -46,9 +46,9 @@ use crate::{ use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; +use datafusion_physical_plan::ExecutionPlanProperties; use bytes::Bytes; -use datafusion_physical_plan::ExecutionPlanProperties; use futures::future::BoxFuture; use futures::{StreamExt, TryStreamExt}; use itertools::Itertools; @@ -363,7 +363,9 @@ impl ExecutionPlan for ParquetExec { 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( + self.properties().output_ordering().is_some(), + ) .repartition_file_groups(&self.base_config.file_groups); let mut new_plan = self.clone(); @@ -1558,7 +1560,7 @@ mod tests { expected_row_num: Option, file_schema: SchemaRef, ) -> Result<()> { - let parquet_exec = Arc::new(ParquetExec::new( + let parquet_exec = ParquetExec::new( FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups, @@ -1571,8 +1573,14 @@ mod tests { }, None, None, - )) as Arc; - assert_eq!(parquet_exec.output_partitioning().partition_count(), 1); + ); + assert_eq!( + parquet_exec + .properties() + .output_partitioning() + .partition_count(), + 1 + ); let results = parquet_exec.execute(0, state.task_ctx())?.next().await; if let Some(expected_row_num) = expected_row_num { diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index dad81b4028bd..eb221a28e2cf 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -55,8 +55,8 @@ use datafusion_physical_expr::{ }; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; - use datafusion_physical_plan::ExecutionPlanProperties; + use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -1404,7 +1404,7 @@ pub(crate) mod tests { // model that it requires the output ordering of its input fn required_input_ordering(&self) -> Vec>> { vec![self - .cache + .properties() .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs)] } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 25280261c0a0..ee5ff7ec59fd 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -64,8 +64,8 @@ use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; - use datafusion_physical_plan::ExecutionPlanProperties; + use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -391,7 +391,7 @@ fn analyze_immediate_sort_removal( // If this sort is unnecessary, we should remove it: if sort_input .equivalence_properties() - .ordering_satisfy(node.plan.output_ordering().unwrap_or(&[])) + .ordering_satisfy(sort_exec.properties().output_ordering().unwrap_or(&[])) { node.plan = if !sort_exec.preserve_partitioning() && sort_input.output_partitioning().partition_count() > 1 diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index da0697eb9aba..7fea375725a5 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -242,7 +242,7 @@ fn require_top_ordering_helper( if children.len() != 1 { Ok((plan, false)) } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let req_ordering = plan.output_ordering().unwrap_or(&[]); + let req_ordering = sort_exec.properties().output_ordering().unwrap_or(&[]); let req_dist = sort_exec.required_input_distribution()[0].clone(); let reqs = PhysicalSortRequirement::from_sort_exprs(req_ordering); Ok(( diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 1b9bf17bffa4..9cb2d6ecbc71 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -52,8 +52,8 @@ use datafusion_physical_expr::{ }; use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; - use datafusion_physical_plan::ExecutionPlanProperties; + use itertools::Itertools; /// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to 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 be4b7d13bc7e..c0abde26c300 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 @@ -31,8 +31,8 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::Transformed; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::tree_node::PlanContext; - use datafusion_physical_plan::ExecutionPlanProperties; + use itertools::izip; /// For a given `plan`, this object carries the information one needs from its diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index 7459deb1f72a..2006402ac59e 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -30,7 +30,6 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_plan::ExecutionPlanProperties; use itertools::Itertools; use std::sync::Arc; @@ -87,7 +86,7 @@ impl TopKAggregation { let children = sort.children(); let child = children.iter().exactly_one().ok()?; - let order = plan.output_ordering()?; + let order = sort.properties().output_ordering()?; let order = order.iter().exactly_one().ok()?; let limit = sort.fetch()?; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index b16cc77a43f2..41a6e4d75be7 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -93,10 +93,10 @@ use datafusion_expr::{ }; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; +use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_sql::utils::window_expr_common_partition_keys; use async_trait::async_trait; -use datafusion_physical_plan::ExecutionPlanProperties; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::{multiunzip, Itertools}; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f96eacf45896..46351a0d430f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -496,7 +496,7 @@ impl AggregateExec { return false; } // ensure there is no output ordering; can this rule be relaxed? - if self.cache.output_ordering().is_some() { + if self.properties().output_ordering().is_some() { return false; } // ensure no ordering is required on the input diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 8036012b7262..45d408bb4de7 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -32,7 +32,7 @@ use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::sorts::sort::{read_spill_as_stream, sort_batch}; use crate::sorts::streaming_merge; use crate::stream::RecordBatchStreamAdapter; -use crate::{aggregates, PhysicalExpr}; +use crate::{aggregates, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; @@ -341,7 +341,7 @@ impl GroupedHashAggregateStream { .with_can_spill(true) .register(context.memory_pool()); let (ordering, _) = agg - .cache + .properties() .equivalence_properties() .find_longest_permutation(&agg_group_by.output_exprs()); let group_ordering = GroupOrdering::try_new( diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 3c5b7e9c13fb..5e7c459a7a98 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -192,7 +192,10 @@ mod tests { let merge = CoalescePartitionsExec::new(csv); // output of CoalescePartitionsExec should have a single partition - assert_eq!(merge.output_partitioning().partition_count(), 1); + assert_eq!( + merge.properties().output_partitioning().partition_count(), + 1 + ); // the result should contain 4 batches (one per input partition) let iter = merge.execute(0, task_ctx)?; diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 38c23331983e..4b7b35e53e1b 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -497,10 +497,6 @@ mod tests { unimplemented!() } - fn output_partitioning(&self) -> &datafusion_physical_expr::Partitioning { - &datafusion_physical_expr::Partitioning::UnknownPartitioning(1) - } - fn children(&self) -> Vec> { vec![] } diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 37faae873745..920e7b9d7a7c 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -26,7 +26,7 @@ use crate::joins::{ }; use crate::memory::MemoryExec; use crate::repartition::RepartitionExec; -use crate::{common, ExecutionPlan, Partitioning}; +use crate::{common, ExecutionPlan, ExecutionPlanProperties, Partitioning}; use arrow::util::pretty::pretty_format_batches; use arrow_array::{ diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index e37f84fb0150..c41d94bf969e 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -101,7 +101,7 @@ pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; /// `ExecutionPlan`'s output from its input. See [`Partitioning`] for more /// details on partitioning. /// -/// Methods such as [`schema`] and [`output_partitioning`] communicate +/// Methods such as [`Self::schema`] and [`ExecutionPlanProperties::output_partitioning`] communicate /// properties of this output to the DataFusion optimizer, and methods such as /// [`required_input_distribution`] and [`required_input_ordering`] express /// requirements of the `ExecutionPlan` from its input. @@ -111,8 +111,6 @@ pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; /// quite verbose) `Debug` output. /// /// [`execute`]: ExecutionPlan::execute -/// [`schema`]: ExecutionPlan::schema -/// [`output_partitioning`]: ExecutionPlan::output_partitioning /// [`required_input_distribution`]: ExecutionPlan::required_input_distribution /// [`required_input_ordering`]: ExecutionPlan::required_input_ordering pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { @@ -455,7 +453,7 @@ impl ExecutionPlanProperties for Arc { /// However, since `ProjectionExec` may calculate derived expressions, it /// needs special handling. /// - /// See also [`Self::maintains_input_order`] and [`Self::output_ordering`] + /// See also [`ExecutionPlan::maintains_input_order`] and [`Self::output_ordering`] /// for related concepts. fn equivalence_properties(&self) -> &EquivalenceProperties { self.properties().equivalence_properties() @@ -605,8 +603,11 @@ impl PlanProperties { /// 2. CoalescePartitionsExec for collapsing all of the partitions into one without ordering guarantee /// 3. SortPreservingMergeExec for collapsing all of the sorted partitions into one with ordering guarantee pub fn need_data_exchange(plan: Arc) -> bool { - if let Some(_) = plan.as_any().downcast_ref::() { - !matches!(plan.output_partitioning(), Partitioning::RoundRobinBatch(_)) + if let Some(repartition) = plan.as_any().downcast_ref::() { + !matches!( + repartition.properties().output_partitioning(), + Partitioning::RoundRobinBatch(_) + ) } else if let Some(coalesce) = plan.as_any().downcast_ref::() { coalesce.input().output_partitioning().partition_count() > 1 @@ -677,10 +678,9 @@ pub fn execute_stream( 1 => plan.execute(0, context), _ => { // merge into a single partition - let plan = Arc::new(CoalescePartitionsExec::new(plan.clone())) - as Arc; + let plan = CoalescePartitionsExec::new(plan.clone()); // CoalescePartitionsExec must produce a single partition - assert_eq!(1, plan.output_partitioning().partition_count()); + assert_eq!(1, plan.properties().output_partitioning().partition_count()); plan.execute(0, context) } } diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 3520d45f9e6c..2d54db3c3cbd 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -410,7 +410,8 @@ impl ExecutionPlan for LocalLimitExec { _ => Statistics { // the result output row number will always be no greater than the limit number num_rows: Precision::Inexact( - self.fetch * self.cache.output_partitioning().partition_count(), + self.fetch + * self.properties().output_partitioning().partition_count(), ), column_statistics: col_stats, diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 23699295e121..04b7c78b77f4 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -287,8 +287,8 @@ mod tests { use std::sync::Arc; use crate::memory::MemoryExec; - use crate::ExecutionPlan; + use crate::ExecutionPlan; use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; @@ -322,8 +322,11 @@ mod tests { let mem_exec = MemoryExec::try_new(&[vec![]], schema, None)? .with_sort_information(sort_information); - assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); - let eq_properties = mem_exec.equivalence_properties(); + assert_eq!( + mem_exec.properties().output_ordering().unwrap(), + expected_output_order + ); + let eq_properties = mem_exec.properties().equivalence_properties(); assert!(eq_properties.oeq_class().contains(&sort1)); assert!(eq_properties.oeq_class().contains(&sort2)); Ok(()) diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index b780a50cdc90..597b60537f6e 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -555,7 +555,7 @@ mod test { let task_ctx = Arc::new(TaskContext::default()); let input = Arc::new(input); - let num_partitions = input.output_partitioning().partition_count(); + let num_partitions = input.properties().output_partitioning().partition_count(); // Configure a RecordBatchReceiverStream to consume all the input partitions let mut builder = diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 9f637361ff8f..2889a506f3ff 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -205,7 +205,7 @@ impl ExecutionPlan for UnionExec { // which is the "meet" of all input orderings. In this example, this // function will return vec![false, true, true], indicating that we // preserve the orderings for the 2nd and the 3rd children. - if let Some(output_ordering) = self.cache.output_ordering() { + if let Some(output_ordering) = self.properties().output_ordering() { self.inputs() .iter() .map(|child| { @@ -635,7 +635,13 @@ mod tests { let union_exec = Arc::new(UnionExec::new(vec![csv, csv2])); // Should have 9 partitions and 9 output batches - assert_eq!(union_exec.output_partitioning().partition_count(), 9); + assert_eq!( + union_exec + .properties() + .output_partitioning() + .partition_count(), + 9 + ); let result: Vec = collect(union_exec, task_ctx).await?; assert_eq!(result.len(), 9); @@ -806,7 +812,7 @@ mod tests { ); let union = UnionExec::new(vec![child1, child2]); - let union_eq_properties = union.equivalence_properties(); + let union_eq_properties = union.properties().equivalence_properties(); let union_actual_orderings = union_eq_properties.oeq_class(); let err_msg = format!( "Error in test id: {:?}, test case: {:?}", From a8fac85429f519ea5ae258a2a6425eaa9ab333c8 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 28 Feb 2024 09:35:09 +0300 Subject: [PATCH 14/15] Bring docs yaml --- .github/workflows/docs.yaml | 64 +++++++++++++++++++++++++++++++++++++ 1 file changed, 64 insertions(+) create mode 100644 .github/workflows/docs.yaml diff --git a/.github/workflows/docs.yaml b/.github/workflows/docs.yaml new file mode 100644 index 000000000000..ab6a615ab60b --- /dev/null +++ b/.github/workflows/docs.yaml @@ -0,0 +1,64 @@ +on: + push: + branches: + - main + paths: + - .asf.yaml + - .github/workflows/docs.yaml + - docs/** + +name: Deploy DataFusion site + +jobs: + build-docs: + name: Build docs + runs-on: ubuntu-latest + steps: + - name: Checkout docs sources + uses: actions/checkout@v4 + + - name: Checkout asf-site branch + uses: actions/checkout@v4 + with: + ref: asf-site + path: asf-site + + - name: Setup Python + uses: actions/setup-python@v5 + with: + python-version: "3.10" + + - name: Install dependencies + run: | + set -x + python3 -m venv venv + source venv/bin/activate + pip install -r docs/requirements.txt + + - name: Build docs + run: | + set -x + source venv/bin/activate + cd docs + ./build.sh + + - name: Copy & push the generated HTML + run: | + set -x + cd asf-site/ + rsync \ + -a \ + --delete \ + --exclude '/.git/' \ + ../docs/build/html/ \ + ./ + cp ../.asf.yaml . + touch .nojekyll + git status --porcelain + if [ "$(git status --porcelain)" != "" ]; then + git config user.name "github-actions[bot]" + git config user.email "github-actions[bot]@users.noreply.github.com" + git add --all + git commit -m 'Publish built docs triggered by ${{ github.sha }}' + git push || git push --force + fi From fea2174574c1a2a24b24479e966fb232bd971435 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 28 Feb 2024 08:08:18 -0800 Subject: [PATCH 15/15] Final reviews and cleanups --- datafusion/core/benches/sort.rs | 19 ++++----- datafusion/core/src/dataframe/mod.rs | 3 +- .../core/src/datasource/listing/table.rs | 7 ++-- datafusion/core/src/datasource/memory.rs | 39 +++++++++++-------- .../core/src/datasource/physical_plan/avro.rs | 6 +-- .../core/src/datasource/physical_plan/csv.rs | 12 +++--- .../core/src/datasource/physical_plan/json.rs | 5 +-- .../datasource/physical_plan/parquet/mod.rs | 5 +-- .../src/physical_optimizer/join_selection.rs | 11 +++--- .../limited_distinct_aggregation.rs | 18 +++++---- .../physical_optimizer/pipeline_checker.rs | 3 +- .../physical_optimizer/projection_pushdown.rs | 3 +- .../src/physical_optimizer/sort_pushdown.rs | 3 +- .../physical_optimizer/topk_aggregation.rs | 5 ++- .../core/src/physical_optimizer/utils.rs | 3 +- datafusion/core/src/physical_planner.rs | 5 +-- datafusion/physical-plan/src/analyze.rs | 8 ++-- .../physical-plan/src/coalesce_partitions.rs | 8 ++-- datafusion/physical-plan/src/common.rs | 3 +- datafusion/physical-plan/src/lib.rs | 18 ++++----- datafusion/physical-plan/src/memory.rs | 2 +- .../physical-plan/src/repartition/mod.rs | 22 +++++------ datafusion/physical-plan/src/stream.rs | 17 ++++---- 23 files changed, 110 insertions(+), 115 deletions(-) diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 34b4a5ebf0dc..94a39bbb2af3 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -68,35 +68,32 @@ use std::sync::Arc; -use arrow::array::DictionaryArray; -use arrow::datatypes::Int32Type; use arrow::{ - array::{Float64Array, Int64Array, StringArray}, + array::{DictionaryArray, Float64Array, Int64Array, StringArray}, compute::SortOptions, - datatypes::Schema, + datatypes::{Int32Type, Schema}, record_batch::RecordBatch, }; -/// Benchmarks for SortPreservingMerge stream -use criterion::{criterion_group, criterion_main, Criterion}; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::{ execution::context::TaskContext, physical_plan::{ - memory::MemoryExec, sorts::sort_preserving_merge::SortPreservingMergeExec, - ExecutionPlan, + coalesce_partitions::CoalescePartitionsExec, memory::MemoryExec, + sorts::sort_preserving_merge::SortPreservingMergeExec, ExecutionPlan, + ExecutionPlanProperties, }, prelude::SessionContext, }; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; + +/// Benchmarks for SortPreservingMerge stream +use criterion::{criterion_group, criterion_main, Criterion}; use futures::StreamExt; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; use tokio::runtime::Runtime; -use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::ExecutionPlanProperties; - /// Total number of streams to divide each input into /// models 8 partition plan (should it be 16??) const NUM_STREAMS: usize = 8; diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 1372570179fe..d7c31b9bd6b3 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -23,7 +23,6 @@ mod parquet; use std::any::Any; use std::sync::Arc; -use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::arrow::record_batch::RecordBatch; use crate::arrow::util::pretty; use crate::datasource::{provider_as_source, MemTable, TableProvider}; @@ -43,7 +42,7 @@ use crate::prelude::SessionContext; use arrow::array::{Array, ArrayRef, Int64Array, StringArray}; use arrow::compute::{cast, concat}; use arrow::csv::WriterBuilder; -use arrow::datatypes::{DataType, Field}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index a1f3d14aacca..00821a1cdd1a 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -27,7 +27,9 @@ use super::PartitionedFile; #[cfg(feature = "parquet")] use crate::datasource::file_format::parquet::ParquetFormat; use crate::datasource::{ - create_ordering, + create_ordering, get_statistics_with_limit, TableProvider, TableType, +}; +use crate::datasource::{ file_format::{ arrow::ArrowFormat, avro::AvroFormat, @@ -36,10 +38,8 @@ use crate::datasource::{ json::JsonFormat, FileFormat, }, - get_statistics_with_limit, listing::ListingTableUrl, physical_plan::{FileScanConfig, FileSinkConfig}, - TableProvider, TableType, }; use crate::{ error::{DataFusionError, Result}, @@ -921,6 +921,7 @@ mod tests { use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::ExecutionPlanProperties; + use tempfile::TempDir; #[tokio::test] diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index cbe20f6a63a1..e47122ccdfda 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -17,35 +17,37 @@ //! [`MemTable`] for querying `Vec` by DataFusion. -use datafusion_physical_plan::metrics::MetricsSet; -use futures::StreamExt; -use log::debug; use std::any::Any; use std::collections::HashMap; use std::fmt::{self, Debug}; use std::sync::Arc; -use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; -use async_trait::async_trait; -use datafusion_common::{not_impl_err, plan_err, Constraints, DFSchema, SchemaExt}; -use datafusion_execution::TaskContext; -use datafusion_physical_plan::ExecutionPlanProperties; -use parking_lot::Mutex; -use tokio::sync::RwLock; -use tokio::task::JoinSet; - use crate::datasource::{TableProvider, TableType}; use crate::error::Result; use crate::execution::context::SessionState; use crate::logical_expr::Expr; use crate::physical_plan::insert::{DataSink, FileSinkExec}; use crate::physical_plan::memory::MemoryExec; -use crate::physical_plan::{common, SendableRecordBatchStream}; -use crate::physical_plan::{repartition::RepartitionExec, Partitioning}; -use crate::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; +use crate::physical_plan::repartition::RepartitionExec; +use crate::physical_plan::{ + common, DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, + Partitioning, SendableRecordBatchStream, +}; use crate::physical_planner::create_physical_sort_expr; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use datafusion_common::{not_impl_err, plan_err, Constraints, DFSchema, SchemaExt}; +use datafusion_execution::TaskContext; +use datafusion_physical_plan::metrics::MetricsSet; + +use async_trait::async_trait; +use futures::StreamExt; +use log::debug; +use parking_lot::Mutex; +use tokio::sync::RwLock; +use tokio::task::JoinSet; + /// Type alias for partition data pub type PartitionData = Arc>>; @@ -362,17 +364,20 @@ impl DataSink for MemSink { #[cfg(test)] mod tests { + use std::collections::HashMap; + use super::*; use crate::datasource::provider_as_source; use crate::physical_plan::collect; use crate::prelude::SessionContext; + use arrow::array::{AsArray, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, UInt64Type}; use arrow::error::ArrowError; use datafusion_common::DataFusionError; use datafusion_expr::LogicalPlanBuilder; + use futures::StreamExt; - use std::collections::HashMap; #[tokio::test] async fn test_with_projection() -> Result<()> { diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 6e7dcf39069c..2ccd83de80cb 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -219,13 +219,15 @@ mod private { #[cfg(test)] #[cfg(feature = "avro")] mod tests { + use super::*; + use crate::arrow::datatypes::{DataType, Field, SchemaBuilder}; use crate::datasource::file_format::{avro::AvroFormat, FileFormat}; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; use crate::prelude::SessionContext; use crate::scalar::ScalarValue; use crate::test::object_store::local_unpartitioned_file; - use arrow::datatypes::{DataType, Field, SchemaBuilder}; + use futures::StreamExt; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; @@ -233,8 +235,6 @@ mod tests { use rstest::*; use url::Url; - use super::*; - #[tokio::test] async fn avro_exec_without_partition() -> Result<()> { test_with_stores(Arc::new(LocalFileSystem::new())).await diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 05a83e8ac0b7..5fcb9f483952 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -32,8 +32,8 @@ use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanProperties, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::csv; @@ -41,7 +41,6 @@ use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_plan::ExecutionPlanProperties; use bytes::{Buf, Bytes}; use futures::{ready, StreamExt, TryStreamExt}; @@ -512,20 +511,23 @@ pub async fn plan_to_csv( #[cfg(test)] mod tests { + use std::fs::{self, File}; + use std::io::Write; + use super::*; use crate::dataframe::DataFrameWriteOptions; use crate::prelude::*; use crate::test::{partitioned_csv_config, partitioned_file_groups}; use crate::{scalar::ScalarValue, test_util::aggr_test_schema}; + use arrow::datatypes::*; use datafusion_common::test_util::arrow_test_data; use datafusion_common::FileType; + use futures::StreamExt; use object_store::chunked::ChunkedStore; use object_store::local::LocalFileSystem; use rstest::*; - use std::fs::{self, File}; - use std::io::Write; use tempfile::TempDir; use url::Url; diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 6f9af2e6abcf..62b96ea3aefb 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -32,15 +32,14 @@ use crate::datasource::physical_plan::FileMeta; use crate::error::{DataFusionError, Result}; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, - PlanProperties, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_plan::ExecutionPlanProperties; use bytes::{Buf, Bytes}; use futures::{ready, StreamExt, TryStreamExt}; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 5ccffde26359..12b62fd68068 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -38,15 +38,14 @@ use crate::{ physical_optimizer::pruning::PruningPredicate, physical_plan::{ metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, Statistics, + DisplayFormatType, ExecutionMode, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }, }; use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; -use datafusion_physical_plan::ExecutionPlanProperties; use bytes::Bytes; use futures::future::BoxFuture; diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 338bf619f9a0..ee60c65ead0b 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -34,16 +34,14 @@ use crate::physical_plan::joins::{ SymmetricHashJoinExec, }; use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow_schema::Schema; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::JoinType; -use datafusion_common::{internal_err, JoinSide}; +use datafusion_common::{internal_err, JoinSide, JoinType}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::sort_properties::SortProperties; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use datafusion_physical_plan::ExecutionPlanProperties; /// The [`JoinSelection`] rule tries to modify a given plan so that it can /// accommodate infinite sources and optimize joins in the plan according to @@ -1366,8 +1364,9 @@ mod util_tests { #[cfg(test)] mod hash_join_tests { - use self::tests_statistical::crosscheck_plans; + use std::sync::Arc; + use self::tests_statistical::crosscheck_plans; use super::*; use crate::physical_optimizer::join_selection::swap_join_type; use crate::physical_optimizer::test_utils::SourceType; @@ -1375,12 +1374,12 @@ mod hash_join_tests { use crate::physical_plan::joins::PartitionMode; use crate::physical_plan::projection::ProjectionExec; use crate::test_util::UnboundedExec; + use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::utils::DataPtr; use datafusion_common::JoinType; use datafusion_physical_plan::ExecutionPlanProperties; - use std::sync::Arc; struct TestCase { case: String, diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index 036c938c1ca6..7be9acec5092 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -18,16 +18,18 @@ //! A special-case optimizer rule that pushes limit into a grouped aggregation //! which has no aggregate expressions or sorting requirements +use std::sync::Arc; + use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::AggregateExec; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; + use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; -use datafusion_physical_plan::ExecutionPlanProperties; + use itertools::Itertools; -use std::sync::Arc; /// An optimizer rule that passes a `limit` hint into grouped aggregations which don't require all /// rows in the group to be processed for correctness. Example queries fitting this description are: @@ -189,6 +191,8 @@ impl PhysicalOptimizerRule for LimitedDistinctAggregation { #[cfg(test)] mod tests { + use std::sync::Arc; + use super::*; use crate::error::Result; use crate::physical_optimizer::aggregate_statistics::tests::TestAggregate; @@ -199,6 +203,7 @@ mod tests { use crate::physical_plan::collect; use crate::physical_plan::memory::MemoryExec; use crate::prelude::SessionContext; + use arrow::array::Int32Array; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; @@ -207,13 +212,10 @@ mod tests { use arrow_schema::SchemaRef; use datafusion_execution::config::SessionConfig; use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::cast; - use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::PhysicalSortExpr; - use datafusion_physical_expr::{expressions, PhysicalExpr}; + use datafusion_physical_expr::expressions::{cast, col}; + use datafusion_physical_expr::{expressions, PhysicalExpr, PhysicalSortExpr}; use datafusion_physical_plan::aggregates::AggregateMode; use datafusion_physical_plan::displayable; - use std::sync::Arc; fn mock_data() -> Result> { let schema = Arc::new(Schema::new(vec![ diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index c0a77eb56f87..e783f75378b1 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -24,14 +24,13 @@ use std::sync::Arc; use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::PhysicalOptimizerRule; -use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use datafusion_common::config::OptimizerOptions; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; -use datafusion_physical_plan::ExecutionPlanProperties; /// The PipelineChecker rule rejects non-runnable query plans that use /// pipeline-breaking operators on infinite input(s). diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 9cb2d6ecbc71..4ed265d59526 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -39,7 +39,7 @@ use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::{Distribution, ExecutionPlan}; +use crate::physical_plan::{Distribution, ExecutionPlan, ExecutionPlanProperties}; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; @@ -52,7 +52,6 @@ use datafusion_physical_expr::{ }; use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; -use datafusion_physical_plan::ExecutionPlanProperties; use itertools::Itertools; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 7c7564fdb400..ff82319fba19 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -28,7 +28,7 @@ use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::tree_node::PlanContext; -use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use datafusion_common::tree_node::Transformed; use datafusion_common::{plan_err, JoinSide, Result}; @@ -37,7 +37,6 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; -use datafusion_physical_plan::ExecutionPlanProperties; /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index 2006402ac59e..0ca709e56bcb 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -17,6 +17,8 @@ //! An optimizer rule that detects aggregate operations that could use a limited bucket count +use std::sync::Arc; + use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::AggregateExec; use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; @@ -24,14 +26,15 @@ use crate::physical_plan::filter::FilterExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::ExecutionPlan; + use arrow_schema::DataType; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; + use itertools::Itertools; -use std::sync::Arc; /// An optimizer rule that passes a `limit` hint to aggregations if the whole result is not needed pub struct TopKAggregation {} diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index f4c2c3873f68..8cc543802e3f 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -25,12 +25,11 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; -use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use datafusion_physical_expr::{LexRequirement, PhysicalSortRequirement}; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::tree_node::PlanContext; -use datafusion_physical_plan::ExecutionPlanProperties; /// This utility function adds a `SortExec` above an operator according to the /// given ordering requirements while preserving the original partitioning. diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 41a6e4d75be7..bf5f5afc5791 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -66,8 +66,8 @@ use crate::physical_plan::unnest::UnnestExec; use crate::physical_plan::values::ValuesExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{ - aggregates, displayable, udaf, windows, AggregateExpr, ExecutionPlan, InputOrderMode, - Partitioning, PhysicalExpr, WindowExpr, + aggregates, displayable, udaf, windows, AggregateExpr, ExecutionPlan, + ExecutionPlanProperties, InputOrderMode, Partitioning, PhysicalExpr, WindowExpr, }; use arrow::compute::SortOptions; @@ -93,7 +93,6 @@ use datafusion_expr::{ }; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; -use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_sql::utils::window_expr_common_partition_keys; use async_trait::async_trait; diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index f771ac238887..5baedc332951 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -25,7 +25,6 @@ use super::{ DisplayAs, Distribution, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, }; - use crate::display::DisplayableExecutionPlan; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; @@ -249,9 +248,7 @@ fn create_output_batch( #[cfg(test)] mod tests { - use arrow::datatypes::{DataType, Field, Schema}; - use futures::FutureExt; - + use super::*; use crate::{ collect, test::{ @@ -260,7 +257,8 @@ mod tests { }, }; - use super::*; + use arrow::datatypes::{DataType, Field, Schema}; + use futures::FutureExt; #[tokio::test] async fn test_drop_cancel() -> Result<()> { diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index d2706cb06f90..1e58260a5344 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -168,10 +168,6 @@ impl ExecutionPlan for CoalescePartitionsExec { #[cfg(test)] mod tests { - - use arrow::datatypes::{DataType, Field, Schema}; - use futures::FutureExt; - use super::*; use crate::test::exec::{ assert_strong_count_converges_to_zero, BlockingExec, PanicExec, @@ -179,6 +175,10 @@ mod tests { use crate::test::{self, assert_is_pending}; use crate::{collect, common}; + use arrow::datatypes::{DataType, Field, Schema}; + + use futures::FutureExt; + #[tokio::test] async fn merge() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 003c60edd9a8..47cdf3e400e3 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -381,11 +381,10 @@ mod tests { use arrow::compute::SortOptions; use arrow::{ - array::{Float32Array, Float64Array}, + array::{Float32Array, Float64Array, UInt64Array}, datatypes::{DataType, Field, Schema}, record_batch::RecordBatch, }; - use arrow_array::UInt64Array; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{col, Column}; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 578ce42f2e9d..b726b587dd1d 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -123,6 +123,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { self.properties().schema().clone() } + /// Gets plan properties, such as output ordering(s), partitioning information etc. fn properties(&self) -> &PlanProperties; /// Specifies the data distribution requirements for all the @@ -400,6 +401,8 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } } +/// This extension trait provides an API to fetch various properties of +/// [`ExecutionPlan`] objects. pub trait ExecutionPlanProperties { fn output_partitioning(&self) -> &Partitioning; @@ -419,21 +422,18 @@ impl ExecutionPlanProperties for Arc { /// Specifies whether this plan generates an infinite stream of records. /// If the plan does not support pipelining, but its input(s) are - /// infinite, returns an error to indicate this. + /// infinite, returns [`ExecutionMode::PipelineBreaking`] to indicate this. fn execution_mode(&self) -> ExecutionMode { self.properties().execution_mode() } /// If the output of this `ExecutionPlan` within each partition is sorted, - /// returns `Some(keys)` with the description of how it was sorted. + /// returns `Some(keys)` describing the ordering. A `None` return value + /// indicates no assumptions should be made on the output ordering. /// - /// For example, Sort, (obviously) produces sorted output as does - /// SortPreservingMergeStream. Less obviously `Projection` - /// produces sorted output if its input was sorted as it does not - /// reorder the input rows, - /// - /// It is safe to return `None` here if your `ExecutionPlan` does not - /// have any particular output order here + /// For example, `SortExec` (obviously) produces sorted output as does + /// `SortPreservingMergeStream`. Less obviously, `Projection` produces sorted + /// output if its input is sorted as it does not reorder the input rows. fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { self.properties().output_ordering() } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index fa2d16530823..ca324a0f7d3b 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -287,8 +287,8 @@ mod tests { use std::sync::Arc; use crate::memory::MemoryExec; - use crate::ExecutionPlan; + use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index d1befb7c53c0..fe93ea131506 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -926,17 +926,7 @@ impl RecordBatchStream for PerPartitionStream { mod tests { use std::collections::HashSet; - use arrow::array::{ArrayRef, StringArray}; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use arrow_array::UInt32Array; - use futures::FutureExt; - use tokio::task::JoinHandle; - - use datafusion_common::cast::as_string_array; - use datafusion_common::{assert_batches_sorted_eq, exec_err}; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; - + use super::*; use crate::{ test::{ assert_is_pending, @@ -948,7 +938,15 @@ mod tests { {collect, expressions::col, memory::MemoryExec}, }; - use super::*; + use arrow::array::{ArrayRef, StringArray, UInt32Array}; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; + use datafusion_common::cast::as_string_array; + use datafusion_common::{assert_batches_sorted_eq, exec_err}; + use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + + use futures::FutureExt; + use tokio::task::JoinHandle; #[tokio::test] async fn one_to_many_round_robin() -> Result<()> { diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index dcba5c74daf1..99d9367740be 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -22,12 +22,14 @@ use std::sync::Arc; use std::task::Context; use std::task::Poll; +use super::metrics::BaselineMetrics; +use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; use crate::displayable; -use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; -use datafusion_common::internal_err; -use datafusion_common::Result; +use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; +use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; + use futures::stream::BoxStream; use futures::{Future, Stream, StreamExt}; use log::debug; @@ -35,9 +37,6 @@ use pin_project_lite::pin_project; use tokio::sync::mpsc::{Receiver, Sender}; use tokio::task::JoinSet; -use super::metrics::BaselineMetrics; -use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; - /// Creates a stream from a collection of producing tasks, routing panics to the stream. /// /// Note that this is similar to [`ReceiverStream` from tokio-stream], with the differences being: @@ -458,13 +457,13 @@ impl futures::Stream for ObservedStream { #[cfg(test)] mod test { use super::*; - use arrow_schema::{DataType, Field, Schema}; - use datafusion_common::exec_err; - use crate::test::exec::{ assert_strong_count_converges_to_zero, BlockingExec, MockExec, PanicExec, }; + use arrow_schema::{DataType, Field, Schema}; + use datafusion_common::exec_err; + fn schema() -> SchemaRef { Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)])) }