diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index d1e4a5b3638e2..59581e91e857a 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -33,15 +33,14 @@ 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::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, 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 +230,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.execution_mode().is_unbounded() { let stream = execute_stream(physical_plan, task_ctx.clone())?; print_options.print_stream(stream, now).await?; } else { @@ -305,10 +304,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 69f9c9530e871..0b7e3d4c64427 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -28,14 +28,14 @@ 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, + Partitioning, PlanProperties, SendableRecordBatchStream, }; use datafusion::prelude::*; use datafusion_expr::{Expr, LogicalPlanBuilder}; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use tokio::time::timeout; @@ -190,6 +190,7 @@ impl TableProvider for CustomDataSource { struct CustomExec { db: CustomDataSource, projected_schema: SchemaRef, + cache: PlanProperties, } impl CustomExec { @@ -199,11 +200,23 @@ impl CustomExec { db: CustomDataSource, ) -> Self { let projected_schema = project_schema(&schema, projections).unwrap(); + let cache = Self::compute_properties(projected_schema.clone()); Self { db, projected_schema, + cache, } } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for CustomExec { @@ -217,16 +230,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index fbb94d66db581..94a39bbb2af31 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -68,34 +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; - /// 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 c04247210d46b..d7c31b9bd6b3d 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -23,15 +23,12 @@ 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}; 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,11 +37,12 @@ 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}; 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; @@ -58,7 +56,6 @@ use datafusion_expr::{ TableProviderFilterPushDown, UNNAMED_TABLE, }; -use crate::prelude::SessionContext; use async_trait::async_trait; /// Contains options that control how data is @@ -1519,7 +1516,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>) { @@ -2907,7 +2904,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(()) } @@ -2956,7 +2953,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 => { @@ -2966,13 +2963,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/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 56e64f556c12e..00821a1cdd1a8 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}, @@ -920,6 +920,8 @@ 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 e087b4bcba511..e47122ccdfda9 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -17,34 +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 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>>; @@ -166,7 +169,7 @@ impl MemTable { // 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)?; @@ -361,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/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index f6c310fb5da15..82774a6e831c9 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,6 +20,8 @@ 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, }; @@ -34,14 +36,13 @@ 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, PlanProperties}; 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)] @@ -52,6 +53,7 @@ pub struct ArrowExec { projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanProperties, } impl ArrowExec { @@ -59,19 +61,53 @@ impl ArrowExec { pub fn new(base_config: FileScanConfig) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = Self::compute_properties( + projected_schema.clone(), + &projected_output_ordering, + &base_config, + ); Self { base_config, projected_schema, projected_statistics, projected_output_ordering, metrics: ExecutionPlanMetricsSet::new(), + cache, } } /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + projected_output_ordering: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = + EquivalenceProperties::new_with_orderings(schema, projected_output_ordering); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + 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(&self.base_config); + self.cache = self.cache.with_partitioning(output_partitioning); + self + } } impl DisplayAs for ArrowExec { @@ -90,25 +126,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -133,12 +152,14 @@ 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.properties().output_ordering().is_some(), + ) .repartition_file_groups(&self.base_config.file_groups); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { let mut new_plan = self.clone(); - new_plan.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 e448bf39f4272..2ccd83de80cb1 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -22,11 +22,10 @@ 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, - Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::datatypes::SchemaRef; @@ -43,6 +42,7 @@ pub struct AvroExec { projected_output_ordering: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanProperties, } impl AvroExec { @@ -50,19 +50,41 @@ impl AvroExec { pub fn new(base_config: FileScanConfig) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = Self::compute_properties( + projected_schema.clone(), + &projected_output_ordering, + &base_config, + ); Self { base_config, projected_schema, projected_statistics, projected_output_ordering, metrics: ExecutionPlanMetricsSet::new(), + cache, } } /// Ref to the base configs pub fn base_config(&self) -> &FileScanConfig { &self.base_config } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + let n_partitions = file_scan_config.file_groups.len(); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(n_partitions), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) + } } 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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -163,6 +168,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}; @@ -213,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; @@ -227,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 @@ -273,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"); @@ -344,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()) @@ -414,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 04959c7904a9a..5fcb9f4839526 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -24,26 +24,25 @@ 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, - Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, 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 bytes::{Buf, Bytes}; -use datafusion_common::config::ConfigOptions; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; @@ -54,8 +53,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, quote: u8, @@ -64,6 +61,7 @@ pub struct CsvExec { metrics: ExecutionPlanMetricsSet, /// Compression type of the file associated with CsvExec pub file_compression_type: FileCompressionType, + cache: PlanProperties, } impl CsvExec { @@ -78,18 +76,21 @@ impl CsvExec { ) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = Self::compute_properties( + projected_schema, + &projected_output_ordering, + &base_config, + ); Self { base_config, - projected_schema, projected_statistics, - projected_output_ordering, has_header, delimiter, quote, escape, metrics: ExecutionPlanMetricsSet::new(), file_compression_type, + cache, } } @@ -115,6 +116,34 @@ impl CsvExec { pub fn escape(&self) -> Option { self.escape } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + 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(&self.base_config); + self.cache = self.cache.with_partitioning(output_partitioning); + self + } } impl DisplayAs for CsvExec { @@ -135,28 +164,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -188,13 +197,15 @@ 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.properties().output_ordering().is_some(), + ) .with_repartition_file_min_size(repartition_file_min_size) .repartition_file_groups(&self.base_config.file_groups); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { let mut new_plan = self.clone(); - new_plan.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) @@ -500,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; @@ -558,7 +572,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 +637,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 +702,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 +765,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 +827,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/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 0d25189a6124d..27022a20e93e9 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 c033c4b89891b..62b96ea3aefbd 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -24,17 +24,16 @@ 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, - Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::json::ReaderBuilder; @@ -44,8 +43,7 @@ use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; 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; @@ -54,11 +52,10 @@ 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: PlanProperties, } impl NdJsonExec { @@ -69,14 +66,17 @@ impl NdJsonExec { ) -> Self { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = Self::compute_properties( + projected_schema, + &projected_output_ordering, + &base_config, + ); Self { base_config, - projected_schema, projected_statistics, - projected_output_ordering, metrics: ExecutionPlanMetricsSet::new(), file_compression_type, + cache, } } @@ -84,6 +84,34 @@ impl NdJsonExec { pub fn base_config(&self) -> &FileScanConfig { &self.base_config } + + fn output_partitioning_helper(file_scan_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_scan_config.file_groups.len()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + file_scan_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + + PlanProperties::new( + eq_properties, + Self::output_partitioning_helper(file_scan_config), // Output Partitioning + 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(&self.base_config); + self.cache = self.cache.with_partitioning(output_partitioning); + self + } } impl DisplayAs for NdJsonExec { @@ -101,26 +129,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -140,7 +150,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.properties().output_ordering().is_some(); let file_groups = &self.base_config.file_groups; let repartitioned_file_groups_option = FileGroupPartitioner::new() @@ -151,7 +161,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))); } @@ -365,11 +375,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; @@ -377,20 +386,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 2a8bb3b4fbaa5..08f1cc9f27265 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, }; @@ -71,11 +69,10 @@ 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 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. @@ -487,20 +484,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 @@ -604,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::{ @@ -611,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 3aa1998bde7e0..12b62fd680680 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, }; @@ -37,16 +38,14 @@ use crate::{ physical_optimizer::pruning::PruningPredicate, physical_plan::{ metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, ExecutionPlan, Partitioning, 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, PhysicalSortExpr, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; 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; @@ -89,8 +88,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, /// Optional predicate for row filtering during parquet scan @@ -103,6 +100,7 @@ pub struct ParquetExec { metadata_size_hint: Option, /// Optional user defined parquet file reader factory parquet_file_reader_factory: Option>, + cache: PlanProperties, } impl ParquetExec { @@ -150,22 +148,25 @@ impl ParquetExec { let (projected_schema, projected_statistics, projected_output_ordering) = base_config.project(); - + let cache = Self::compute_properties( + projected_schema, + &projected_output_ordering, + &base_config, + ); 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, predicate, pruning_predicate, page_pruning_predicate, metadata_size_hint, parquet_file_reader_factory: None, + cache, } } @@ -260,6 +261,34 @@ impl ParquetExec { self.enable_bloom_filter .unwrap_or(config_options.execution.parquet.bloom_filter_enabled) } + + fn output_partitioning_helper(file_config: &FileScanConfig) -> Partitioning { + Partitioning::UnknownPartitioning(file_config.file_groups.len()) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + file_config: &FileScanConfig, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + + PlanProperties::new( + eq_properties, + 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(&self.base_config); + self.cache = self.cache.with_partitioning(output_partitioning); + self + } } impl DisplayAs for ParquetExec { @@ -306,8 +335,8 @@ impl ExecutionPlan for ParquetExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.projected_schema) + fn properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -315,24 +344,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>, @@ -351,12 +362,14 @@ 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.properties().output_ordering().is_some(), + ) .repartition_file_groups(&self.base_config.file_groups); let mut new_plan = self.clone(); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - new_plan.base_config.file_groups = repartitioned_file_groups; + new_plan = new_plan.with_file_groups(repartitioned_file_groups); } Ok(Some(Arc::new(new_plan))) } @@ -763,6 +776,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; @@ -780,28 +795,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; @@ -1561,7 +1573,13 @@ mod tests { None, None, ); - 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 { @@ -1675,7 +1693,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 4f8806a685923..eb221a28e2cf1 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -54,8 +54,8 @@ 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 datafusion_physical_plan::ExecutionPlanProperties; use itertools::izip; @@ -427,31 +427,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) } + + join_plan.children[0].data = left_keys; + join_plan.children[1].data = right_keys; + Ok(join_plan) } fn reorder_aggregate_keys( @@ -605,32 +601,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, - )?)); - } + left.equivalence_properties(), + right.equivalence_properties(), + ); + 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 { @@ -644,24 +636,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() { + left.equivalence_properties(), + right.equivalence_properties(), + ); + 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(), @@ -682,28 +672,28 @@ 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()) { +) -> (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) + try_reorder(join_keys, right_exprs, right_equivalence_properties) } - _ => None, + _ => (join_keys, None), } } @@ -711,66 +701,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. @@ -883,12 +872,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() - }); + .satisfy(&dist, input.plan.equivalence_properties()); // Add hash repartitioning when: // - The hash distribution requirement is not satisfied, or @@ -901,7 +889,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 _; @@ -1077,7 +1065,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.execution_mode().is_unbounded(); // Use order preserving variants either of the conditions true // - it is desired according to config // - when plan is unbounded @@ -1312,8 +1300,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::{ @@ -1344,6 +1331,7 @@ pub(crate) mod tests { expressions, expressions::binary, expressions::lit, expressions::Column, LexOrdering, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; + use datafusion_physical_plan::PlanProperties; /// Models operators like BoundedWindowExec that require an input /// ordering but is easy to construct @@ -1351,23 +1339,35 @@ pub(crate) mod tests { struct SortRequiredExec { input: Arc, expr: LexOrdering, + cache: PlanProperties, } 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 = Self::compute_properties(&input); Self { input, expr: requirement, + cache, } } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + 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 + ) + } } impl DisplayAs for SortRequiredExec { @@ -1389,22 +1389,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 properties(&self) -> &PlanProperties { + &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()] } @@ -1412,6 +1404,7 @@ pub(crate) mod tests { // model that it requires the output ordering of its input fn required_input_ordering(&self) -> Vec>> { vec![self + .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 e6f3e12aceaf2..5fac1397e023d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -64,7 +64,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; 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 datafusion_physical_plan::ExecutionPlanProperties; use itertools::izip; @@ -208,7 +208,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.execution_mode().is_unbounded() { return Ok(plan); } @@ -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(sort_exec.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 @@ -574,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(), + 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 f2e81fb053c09..ee60c65ead0b2 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::{ @@ -37,12 +34,11 @@ 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}; @@ -231,7 +227,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 +236,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 +251,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 +441,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 +459,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.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. @@ -550,10 +542,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,15 +590,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; - if left_unbounded - && !right_unbounded +) -> Result> { + if let Some(hash_join) = input.as_any().downcast_ref::() { + if hash_join.left.execution_mode().is_unbounded() + && !hash_join.right.execution_mode().is_unbounded() && matches!( *hash_join.join_type(), JoinType::Inner @@ -618,7 +604,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 +640,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 +656,6 @@ mod tests_statistical { use super::*; use crate::{ - physical_optimizer::test_utils::check_integrity, physical_plan::{ displayable, joins::PartitionMode, ColumnStatistics, Statistics, }, @@ -829,19 +804,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, @@ -1391,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; @@ -1400,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::empty::EmptyExec; - use std::sync::Arc; + use datafusion_physical_plan::ExecutionPlanProperties; struct TestCase { case: String, @@ -1772,18 +1746,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 +1778,12 @@ mod hash_join_tests { assert_eq!( ( t.case.as_str(), - if left.unbounded_output(&[])? { + if left.execution_mode().is_unbounded() { SourceType::Unbounded } else { SourceType::Bounded }, - if right.unbounded_output(&[])? { + if right.execution_mode().is_unbounded() { SourceType::Unbounded } else { SourceType::Bounded diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index 9855247151b88..7be9acec50920 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -18,15 +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 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: @@ -188,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; @@ -198,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}; @@ -206,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/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 4d03840d3dd31..7fea375725a54 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::{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 @@ -92,6 +90,7 @@ pub(crate) struct OutputRequirementExec { input: Arc, order_requirement: Option, dist_requirement: Distribution, + cache: PlanProperties, } impl OutputRequirementExec { @@ -100,16 +99,27 @@ impl OutputRequirementExec { requirements: Option, dist_requirement: Distribution, ) -> Self { + let cache = Self::compute_properties(&input); Self { input, order_requirement: requirements, dist_requirement, + cache, } } pub(crate) fn input(&self) -> Arc { self.input.clone() } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + 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 + ) + } } impl DisplayAs for OutputRequirementExec { @@ -127,12 +137,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 properties(&self) -> &PlanProperties { + &self.cache } fn benefits_from_input_partitioning(&self) -> Vec { @@ -143,10 +149,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 +161,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>, @@ -245,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 = 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/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index c0f071cd3f646..e783f75378b1e 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::tree_node::PlanContext; /// The PipelineChecker rule rejects non-runnable query plans that use /// pipeline-breaking operators on infinite input(s). @@ -51,10 +50,7 @@ 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) + plan.transform_up(&|p| check_finiteness_requirements(p, &config.optimizer)) } fn name(&self) -> &str { @@ -66,21 +62,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 +76,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.execution_mode().pipeline_friendly() { + plan_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 +130,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 +155,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 +205,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 +225,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 +249,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 +272,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 +304,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/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 79d22374f9c24..4ed265d59526e 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; 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 bc9bd0010dc58..c0abde26c300a 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,7 @@ 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 datafusion_physical_plan::ExecutionPlanProperties; use itertools::izip; @@ -120,7 +120,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 +176,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 +241,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.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_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 22e0d804acb1f..ff82319fba195 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}; @@ -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.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 dd02614203043..0ca709e56bcbc 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 {} @@ -86,7 +89,7 @@ impl TopKAggregation { let children = sort.children(); let child = children.iter().exactly_one().ok()?; - let order = sort.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_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 4f4b17345ef8b..8cc543802e3f4 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -25,7 +25,7 @@ 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}; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 83ba773464f6e..bf5f5afc5791f 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; @@ -1990,30 +1990,36 @@ 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, + PlanProperties, 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 fmt::Debug; - use std::collections::HashMap; - use std::convert::TryFrom; - use std::ops::{BitAnd, Not}; - use std::{any::Any, fmt}; + use datafusion_physical_expr::EquivalenceProperties; fn make_session_state() -> SessionState { let runtime = Arc::new(RuntimeEnv::default()); @@ -2575,7 +2581,26 @@ mod tests { #[derive(Debug)] struct NoOpExecutionPlan { - schema: SchemaRef, + cache: PlanProperties, + } + + impl NoOpExecutionPlan { + fn new(schema: SchemaRef) -> Self { + 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 compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + // Output Partitioning + Partitioning::UnknownPartitioning(1), + // Execution Mode + ExecutionMode::Bounded, + ) + } } impl DisplayAs for NoOpExecutionPlan { @@ -2594,16 +2619,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -2641,13 +2658,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 ed5aa15e291b5..0042554f6c734 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -42,9 +42,11 @@ 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}; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionMode, PlanProperties, +}; #[cfg(feature = "compression")] use bzip2::write::BzEncoder; @@ -365,18 +367,34 @@ pub fn csv_exec_ordered( pub struct StatisticsExec { stats: Statistics, schema: Arc, + cache: PlanProperties, } + impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { assert_eq!( stats.column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); + let cache = Self::compute_properties(Arc::new(schema.clone())); Self { stats, schema: Arc::new(schema), + cache, } } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + // Output Partitioning + Partitioning::UnknownPartitioning(2), + // Execution Mode + ExecutionMode::Bounded, + ) + } } impl DisplayAs for StatisticsExec { @@ -403,16 +421,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 properties(&self) -> &PlanProperties { + &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 282b0f7079ee2..3244ad49d1c6f 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, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, }; use crate::prelude::{CsvReadOptions, SessionContext}; @@ -47,19 +46,17 @@ 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 datafusion_physical_expr::EquivalenceProperties; 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}; - /// Scan an empty data source, mainly used in tests pub fn scan_empty( name: Option<&str>, @@ -230,7 +227,7 @@ impl TableProvider for TestTableProvider { pub struct UnboundedExec { batch_produce: Option, batch: RecordBatch, - partitions: usize, + cache: PlanProperties, } impl UnboundedExec { /// Create new exec that clones the given record batch to its output. @@ -241,12 +238,32 @@ impl UnboundedExec { batch: RecordBatch, partitions: usize, ) -> Self { + let cache = Self::compute_properties(batch.schema(), batch_produce, partitions); Self { batch_produce, batch, - partitions, + cache, } } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + batch_produce: Option, + n_partitions: usize, + ) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + let mode = if batch_produce.is_none() { + ExecutionMode::Unbounded + } else { + ExecutionMode::Bounded + }; + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(n_partitions), + mode, + ) + } } impl DisplayAs for UnboundedExec { @@ -272,19 +289,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index a9ea5cc2a35c8..aa3f35e295415 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,16 @@ 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, PlanProperties}; 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 +68,41 @@ macro_rules! TEST_CUSTOM_RECORD_BATCH { }; } +//--- Custom source dataframe tests ---// + +struct CustomTableProvider; +#[derive(Debug, Clone)] +struct CustomExecutionPlan { + projection: Option>, + cache: PlanProperties, +} + +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 = 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 compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + // Output Partitioning + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } +} + +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 +144,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -197,9 +213,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 bc7f88b39672e..bc6d85a74a512 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -25,17 +25,17 @@ 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, - Statistics, + DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanProperties, SendableRecordBatchStream, Statistics, }; use datafusion::prelude::*; use datafusion::scalar::ScalarValue; use datafusion_common::cast::as_primitive_array; use datafusion_common::{internal_err, not_impl_err}; use datafusion_expr::expr::{BinaryExpr, Cast}; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; @@ -57,8 +57,25 @@ fn create_batch(value: i32, num_rows: usize) -> Result { #[derive(Debug)] struct CustomPlan { - schema: SchemaRef, batches: Vec, + cache: PlanProperties, +} + +impl CustomPlan { + fn new(schema: SchemaRef, batches: Vec) -> Self { + 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 compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for CustomPlan { @@ -80,16 +97,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -183,25 +192,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 f0985f5546543..85ac47dc97fc9 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, ExecutionMode, ExecutionPlan, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }, prelude::SessionContext, scalar::ScalarValue, }; +use datafusion_common::{project_schema, stats::Precision}; +use datafusion_physical_expr::EquivalenceProperties; 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: PlanProperties, } impl StatisticsValidation { @@ -51,7 +53,23 @@ impl StatisticsValidation { schema.fields().len(), "the column statistics vector length should be the number of fields" ); - Self { stats, schema } + let cache = Self::compute_properties(schema.clone()); + Self { + stats, + schema, + cache, + } + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(2), + ExecutionMode::Bounded, + ) } } @@ -131,16 +149,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 properties(&self) -> &PlanProperties { + &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 29708c4422cac..2c12e108bb47b 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}, @@ -81,9 +81,10 @@ use datafusion::{ UserDefinedLogicalNodeCore, }, optimizer::{optimize_children, OptimizerConfig, OptimizerRule}, + physical_expr::EquivalenceProperties, physical_plan::{ - expressions::PhysicalSortExpr, DisplayAs, DisplayFormatType, Distribution, - ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, + DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, + Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }, physical_planner::{DefaultPhysicalPlanner, ExtensionPlanner, PhysicalPlanner}, @@ -91,7 +92,6 @@ use datafusion::{ }; 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,25 @@ struct TopKExec { input: Arc, /// The maxium number of values k: usize, + cache: PlanProperties, +} + +impl TopKExec { + fn new(input: Arc, k: usize) -> Self { + 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 compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } } impl Debug for TopKExec { @@ -441,16 +460,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 properties(&self) -> &PlanProperties { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -465,10 +476,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-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 301f12e9aa2ea..a4a246daf6760 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 855408c4baa80..65987e01553d8 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, ExecutionPlanProperties, PlanProperties}; 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, Result}; +use datafusion_common::{internal_err, not_impl_err, 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; @@ -260,16 +260,12 @@ 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, /// 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: PlanProperties, } impl AggregateExec { @@ -344,7 +340,7 @@ impl AggregateExec { &new_requirement, &mut aggr_expr, &group_by, - &input_eq_properties, + input_eq_properties, &mode, )?; new_requirement.extend(req); @@ -366,10 +362,13 @@ 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(); - + let cache = Self::compute_properties( + &input, + schema.clone(), + &projection_mapping, + &mode, + &input_order_mode, + ); Ok(AggregateExec { mode, group_by, @@ -378,12 +377,11 @@ impl AggregateExec { input, schema, input_schema, - projection_mapping, metrics: ExecutionPlanMetricsSet::new(), required_input_ordering, limit: None, input_order_mode, - output_ordering, + cache, }) } @@ -498,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.properties().output_ordering().is_some() { return false; } // ensure no ordering is required on the input @@ -507,6 +505,54 @@ impl AggregateExec { } true } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + schema: SchemaRef, + projection_mapping: &ProjectionMapping, + mode: &AggregateMode, + input_order_mode: &InputOrderMode, + ) -> PlanProperties { + // Construct equivalence properties: + let eq_properties = input + .equivalence_properties() + .project(projection_mapping, schema); + + // Get output partitioning: + 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 = input.equivalence_properties(); + if let Partitioning::Hash(exprs, part) = output_partitioning { + let normalized_exprs = exprs + .iter() + .map(|expr| { + input_eq_properties + .project_expr(expr, projection_mapping) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) + }) + .collect(); + output_partitioning = Partitioning::Hash(normalized_exprs, part); + } + } + + // Determine execution mode: + let mut exec_mode = input.execution_mode(); + if exec_mode == ExecutionMode::Unbounded + && *input_order_mode == InputOrderMode::Linear + { + // Cannot run without breaking the pipeline + exec_mode = ExecutionMode::PipelineBreaking; + } + + PlanProperties::new(eq_properties, output_partitioning, exec_mode) + } + 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 properties(&self) -> &PlanProperties { + &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,27 @@ mod tests { struct TestYieldingExec { /// True if this exec should yield back to runtime the first time it is polled pub yield_first: bool, + cache: PlanProperties, + } + + impl TestYieldingExec { + fn new(yield_first: bool) -> Self { + let schema = some_data().0; + 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 compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + // Output Partitioning + Partitioning::UnknownPartitioning(1), + // Execution Mode + ExecutionMode::Bounded, + ) + } } impl DisplayAs for TestYieldingExec { @@ -1650,16 +1661,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -1741,72 +1745,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/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index f9db0a050cfc6..45d408bb4de7d 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -341,6 +341,7 @@ impl GroupedHashAggregateStream { .with_can_spill(true) .register(context.memory_pool()); let (ordering, _) = agg + .properties() .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 4f1578e220ddd..5baedc332951e 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -20,16 +20,18 @@ 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, ExecutionPlanProperties, PlanProperties, + SendableRecordBatchStream, +}; use crate::display::DisplayableExecutionPlan; 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; @@ -45,6 +47,7 @@ pub struct AnalyzeExec { pub(crate) input: Arc, /// The output schema for RecordBatches of this exec node schema: SchemaRef, + cache: PlanProperties, } impl AnalyzeExec { @@ -55,11 +58,13 @@ impl AnalyzeExec { input: Arc, schema: SchemaRef, ) -> Self { + let cache = Self::compute_properties(&input, schema.clone()); AnalyzeExec { verbose, show_statistics, input, schema, + cache, } } @@ -77,6 +82,17 @@ impl AnalyzeExec { pub fn input(&self) -> &Arc { &self.input } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + schema: SchemaRef, + ) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + let output_partitioning = Partitioning::UnknownPartitioning(1); + let exec_mode = input.execution_mode(); + PlanProperties::new(eq_properties, output_partitioning, exec_mode) + } } impl DisplayAs for AnalyzeExec { @@ -99,8 +115,8 @@ impl ExecutionPlan for AnalyzeExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &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>, @@ -252,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::{ @@ -263,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_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 09d1ea87ca370..0b9ecebbb1e8c 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, ExecutionPlanProperties, PlanProperties, 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,15 +48,18 @@ pub struct CoalesceBatchesExec { target_batch_size: usize, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanProperties, } impl CoalesceBatchesExec { /// Create a new CoalesceBatchesExec pub fn new(input: Arc, target_batch_size: usize) -> Self { + let cache = Self::compute_properties(&input); Self { input, target_batch_size, metrics: ExecutionPlanMetricsSet::new(), + cache, } } @@ -72,6 +72,17 @@ impl CoalesceBatchesExec { pub fn target_batch_size(&self) -> usize { self.target_batch_size } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(input: &Arc) -> PlanProperties { + // The coalesce batches operator does not make any changes to the + // partitioning of its input. + PlanProperties::new( + input.equivalence_properties().clone(), // Equivalence Properties + input.output_partitioning().clone(), // Output Partitioning + input.execution_mode(), // Execution Mode + ) + } } impl DisplayAs for CoalesceBatchesExec { @@ -98,34 +109,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 properties(&self) -> &PlanProperties { + &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 +125,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 +289,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 1521daee1334a..1e58260a5344f 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -21,17 +21,17 @@ 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, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, + Statistics, +}; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; -use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, 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,14 +41,17 @@ pub struct CoalescePartitionsExec { input: Arc, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanProperties, } impl CoalescePartitionsExec { /// Create a new CoalescePartitionsExec pub fn new(input: Arc) -> Self { + let cache = Self::compute_properties(&input); CoalescePartitionsExec { input, metrics: ExecutionPlanMetricsSet::new(), + cache, } } @@ -56,6 +59,19 @@ impl CoalescePartitionsExec { pub fn input(&self) -> &Arc { &self.input } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(input: &Arc) -> PlanProperties { + // Coalescing partitions loses existing orderings: + let mut eq_properties = input.equivalence_properties().clone(); + eq_properties.clear_orderings(); + + PlanProperties::new( + eq_properties, // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + input.execution_mode(), // Execution Mode + ) + } } impl DisplayAs for CoalescePartitionsExec { @@ -78,36 +94,14 @@ impl ExecutionPlan for CoalescePartitionsExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() + fn properties(&self) -> &PlanProperties { + &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] } @@ -174,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, @@ -185,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()); @@ -198,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/common.rs b/datafusion/physical-plan/src/common.rs index 5172bc9b2a3c7..47cdf3e400e36 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}; @@ -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/display.rs b/datafusion/physical-plan/src/display.rs index ff106dceb974d..4b7b35e53e1b0 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 crate::{DisplayAs, ExecutionPlan}; - use super::DisplayableExecutionPlan; + use crate::{DisplayAs, ExecutionPlan, PlanProperties}; + + use datafusion_common::{DataFusionError, Result, Statistics}; + use datafusion_execution::{SendableRecordBatchStream, TaskContext}; #[derive(Debug, Clone, Copy)] enum TestStatsExecPlan { @@ -496,18 +493,8 @@ 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 output_ordering( - &self, - ) -> Option<&[datafusion_physical_expr::PhysicalSortExpr]> { - None + fn properties(&self) -> &PlanProperties { + unimplemented!() } fn children(&self) -> Vec> { @@ -517,28 +504,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 398edc211de3f..ebd92efb4cd22 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -20,14 +20,17 @@ use std::any::Any; use std::sync::Arc; -use super::expressions::PhysicalSortExpr; -use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; +use super::{ + common, DisplayAs, ExecutionMode, PlanProperties, SendableRecordBatchStream, + Statistics, +}; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use log::trace; @@ -38,26 +41,49 @@ pub struct EmptyExec { schema: SchemaRef, /// Number of partitions partitions: usize, + cache: PlanProperties, } impl EmptyExec { /// Create a new EmptyExec pub fn new(schema: SchemaRef) -> Self { + let cache = Self::compute_properties(schema.clone(), 1); EmptyExec { schema, partitions: 1, + 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(self.partitions); + self.cache = self.cache.with_partitioning(output_partitioning); self } fn data(&self) -> Result> { Ok(vec![]) } + + fn output_partitioning_helper(n_partitions: usize) -> Partitioning { + Partitioning::UnknownPartitioning(n_partitions) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef, n_partitions: usize) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + let output_partitioning = Self::output_partitioning_helper(n_partitions); + PlanProperties::new( + eq_properties, + // Output Partitioning + output_partitioning, + // Execution Mode + ExecutionMode::Bounded, + ) + } } impl DisplayAs for EmptyExec { @@ -80,23 +106,14 @@ impl ExecutionPlan for EmptyExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &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 babcaf078bdcd..320ee37bed954 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, PlanProperties, SendableRecordBatchStream}; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; @@ -29,6 +28,7 @@ use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatc use datafusion_common::display::StringifiedPlan; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use log::trace; @@ -43,6 +43,7 @@ pub struct ExplainExec { stringified_plans: Vec, /// control which plans to print verbose: bool, + cache: PlanProperties, } impl ExplainExec { @@ -52,10 +53,12 @@ impl ExplainExec { stringified_plans: Vec, verbose: bool, ) -> Self { + let cache = Self::compute_properties(schema.clone()); ExplainExec { schema, stringified_plans, verbose, + cache, } } @@ -68,6 +71,16 @@ impl ExplainExec { pub fn verbose(&self) -> bool { self.verbose } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for ExplainExec { @@ -90,8 +103,8 @@ impl ExecutionPlan for ExplainExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -99,15 +112,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 362fa10efc9f5..4155b00820f42 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, ExecutionPlanProperties, PlanProperties, + 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: PlanProperties, } impl FilterExec { @@ -72,12 +72,18 @@ 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 default_selectivity = 20; + let cache = + Self::compute_properties(&input, &predicate, default_selectivity)?; + Ok(Self { + predicate, + input: input.clone(), + metrics: ExecutionPlanMetricsSet::new(), + default_selectivity, + cache, + }) + } other => { plan_err!("Filter predicate must return boolean values, not {other:?}") } @@ -109,6 +115,80 @@ impl FilterExec { pub fn default_selectivity(&self) -> u8 { self.default_selectivity } + + /// Calculates `Statistics` for `FilterExec`, by applying selectivity (either default, or estimated) to input statistics. + 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, + }) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + 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_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 _; + eq_properties.add_equal_conditions(&lhs_expr, &rhs_expr) + } + // Add the columns that have only one viable value (singleton) after + // filtering to constants. + 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); + + Ok(PlanProperties::new( + eq_properties, + input.output_partitioning().clone(), // Output Partitioning + input.execution_mode(), // Execution Mode + )) + } } impl DisplayAs for FilterExec { @@ -131,55 +211,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 properties(&self) -> &PlanProperties { + &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>, @@ -214,43 +258,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 e678425d37534..16c929b78144d 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, ExecutionPlanProperties, Partitioning, + PlanProperties, SendableRecordBatchStream, }; use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; @@ -35,7 +35,9 @@ use arrow_array::{ArrayRef, UInt64Array}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{exec_err, internal_err, 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; @@ -86,6 +88,7 @@ pub struct FileSinkExec { count_schema: SchemaRef, /// Optional required sort order for output data. sort_order: Option>, + cache: PlanProperties, } impl fmt::Debug for FileSinkExec { @@ -102,12 +105,15 @@ impl FileSinkExec { sink_schema: SchemaRef, sort_order: Option>, ) -> Self { + let count_schema = make_count_schema(); + let cache = Self::create_schema(&input, count_schema); Self { input, sink, sink_schema, count_schema: make_count_schema(), sort_order, + cache, } } @@ -170,6 +176,18 @@ impl FileSinkExec { pub fn metrics(&self) -> Option { self.sink.metrics() } + + fn create_schema( + input: &Arc, + schema: SchemaRef, + ) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + input.execution_mode(), + ) + } } 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 properties(&self) -> &PlanProperties { + &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 99bd051da0f79..9f8dc0ce56b0a 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -23,23 +23,24 @@ 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::DisplayAs; +use crate::ExecutionPlanProperties; use crate::{ - coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, - ColumnStatistics, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + execution_mode_from_children, ColumnStatistics, DisplayAs, DisplayFormatType, + Distribution, ExecutionMode, ExecutionPlan, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; 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, 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 +62,7 @@ pub struct CrossJoinExec { left_fut: OnceAsync, /// Execution plan metrics metrics: ExecutionPlanMetricsSet, + cache: PlanProperties, } impl CrossJoinExec { @@ -76,13 +78,14 @@ impl CrossJoinExec { }; let schema = Arc::new(Schema::new(all_columns)); - + let cache = Self::compute_properties(&left, &right, schema.clone()); CrossJoinExec { left, right, schema, left_fut: Default::default(), metrics: ExecutionPlanMetricsSet::default(), + cache, } } @@ -95,6 +98,43 @@ impl CrossJoinExec { pub fn right(&self) -> &Arc { &self.right } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + left: &Arc, + right: &Arc, + schema: SchemaRef, + ) -> PlanProperties { + // Calculate equivalence properties + // TODO: Check equivalence properties of cross join, it may preserve + // ordering in some cases. + let eq_properties = join_equivalence_properties( + left.equivalence_properties().clone(), + right.equivalence_properties().clone(), + &JoinType::Full, + schema, + &[false, false], + None, + &[], + ); + + // Get output partitioning: + // TODO: Optimize the cross join implementation to generate M * N + // partitions. + let output_partitioning = adjust_right_output_partitioning( + right.output_partitioning(), + left.schema().fields.len(), + ); + + // Determine the execution mode: + 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; + } + + PlanProperties::new(eq_properties, output_partitioning, mode) + } } /// Asynchronously collect the result of the left child @@ -158,8 +198,8 @@ impl ExecutionPlan for CrossJoinExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -170,19 +210,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 +227,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 cd8b17d135986..ee3438c6a3635 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -23,29 +23,27 @@ 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, - calculate_join_output_ordering, get_final_indices_from_bit_map, - need_produce_result_in_final, JoinHashMap, JoinHashMapOffset, JoinHashMapType, +use super::{ + utils::{OnceAsync, OnceFut}, + PartitionMode, }; +use crate::ExecutionPlanProperties; use crate::{ coalesce_partitions::CoalescePartitionsExec, - expressions::PhysicalSortExpr, + execution_mode_from_children, 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, JoinOnRef, StatefulStreamResult, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, -}; -use crate::{handle_state, DisplayAs}; - -use super::{ - utils::{OnceAsync, OnceFut}, - PartitionMode, + DisplayAs, DisplayFormatType, Distribution, ExecutionMode, ExecutionPlan, + Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, + Statistics, }; use arrow::array::{ @@ -66,7 +64,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 +286,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 +297,8 @@ 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: PlanProperties, } impl HashJoinExec { @@ -330,14 +328,13 @@ 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(&[]), + let cache = Self::compute_properties( + &left, + &right, + Arc::new(schema.clone()), *join_type, &on, - left_schema.fields.len(), - &Self::maintains_input_order(*join_type), - Some(Self::probe_side()), + partition_mode, ); Ok(HashJoinExec { @@ -353,7 +350,7 @@ impl HashJoinExec { metrics: ExecutionPlanMetricsSet::new(), column_indices, null_equals_null, - output_order, + cache, }) } @@ -408,6 +405,77 @@ impl HashJoinExec { // In current implementation right side is always probe side. JoinSide::Right } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + left: &Arc, + right: &Arc, + schema: SchemaRef, + join_type: JoinType, + on: JoinOnRef, + mode: PartitionMode, + ) -> PlanProperties { + // Calculate equivalence properties: + let eq_properties = join_equivalence_properties( + left.equivalence_properties().clone(), + right.equivalence_properties().clone(), + &join_type, + schema, + &Self::maintains_input_order(join_type), + Some(Self::probe_side()), + on, + ); + + // Get output partitioning: + let left_columns_len = left.schema().fields.len(); + let output_partitioning = match mode { + PartitionMode::CollectLeft => match join_type { + JoinType::Inner | JoinType::Right => adjust_right_output_partitioning( + right.output_partitioning(), + 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( + join_type, + left.output_partitioning(), + right.output_partitioning(), + left_columns_len, + ), + PartitionMode::Auto => Partitioning::UnknownPartitioning( + right.output_partitioning().partition_count(), + ), + }; + + // 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 `Left`, `Full`, `LeftAnti` or `LeftSemi` join types. + let pipeline_breaking = left.execution_mode().is_unbounded() + || (right.execution_mode().is_unbounded() + && matches!( + join_type, + JoinType::Left + | JoinType::Full + | JoinType::LeftAnti + | JoinType::LeftSemi + )); + + let mode = if pipeline_breaking { + ExecutionMode::PipelineBreaking + } else { + execution_mode_from_children([left, right]) + }; + + PlanProperties::new(eq_properties, output_partitioning, mode) + } } impl DisplayAs for HashJoinExec { @@ -439,8 +507,8 @@ impl ExecutionPlan for HashJoinExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -464,71 +532,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 +552,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 1618efd4d0f97..c6d891dd13c1a 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -34,7 +34,8 @@ use crate::joins::utils::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + execution_mode_from_children, DisplayAs, DisplayFormatType, Distribution, + ExecutionMode, ExecutionPlan, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, }; @@ -49,7 +50,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 +93,8 @@ pub struct NestedLoopJoinExec { column_indices: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl NestedLoopJoinExec { @@ -108,15 +110,18 @@ impl NestedLoopJoinExec { check_join_is_valid(&left_schema, &right_schema, &[])?; let (schema, column_indices) = build_join_schema(&left_schema, &right_schema, join_type); + let schema = Arc::new(schema); + let cache = Self::compute_properties(&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, }) } @@ -139,6 +144,46 @@ impl NestedLoopJoinExec { pub fn join_type(&self) -> &JoinType { &self.join_type } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + left: &Arc, + right: &Arc, + schema: SchemaRef, + join_type: JoinType, + ) -> PlanProperties { + // Calculate equivalence properties: + let eq_properties = join_equivalence_properties( + 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 join_type == JoinType::Full { + left.output_partitioning().clone() + } else { + partitioned_join_output_partitioning( + join_type, + left.output_partitioning(), + right.output_partitioning(), + left.schema().fields.len(), + ) + }; + + // Determine execution mode: + let mut mode = execution_mode_from_children([left, right]); + if mode.is_unbounded() { + mode = ExecutionMode::PipelineBreaking; + } + + PlanProperties::new(eq_properties, output_partitioning, mode) + } } impl DisplayAs for NestedLoopJoinExec { @@ -164,46 +209,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 properties(&self) -> &PlanProperties { + &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 +767,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 7af614e534917..7b70a2952b4c3 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -32,18 +32,18 @@ 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, JoinOnRef, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ - metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + execution_mode_from_children, metrics, DisplayAs, DisplayFormatType, Distribution, + ExecutionPlan, ExecutionPlanProperties, PhysicalExpr, PlanProperties, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; 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; @@ -53,9 +53,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 +79,12 @@ 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 holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl SortMergeJoinExec { @@ -137,19 +135,10 @@ 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 = + Self::compute_properties(&left, &right, schema.clone(), join_type, &on); Ok(Self { left, right, @@ -160,9 +149,9 @@ impl SortMergeJoinExec { metrics: ExecutionPlanMetricsSet::new(), left_sort_exprs, right_sort_exprs, - output_ordering, sort_options, null_equals_null, + cache, }) } @@ -200,16 +189,50 @@ 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. + fn compute_properties( + left: &Arc, + right: &Arc, + schema: SchemaRef, + join_type: JoinType, + join_on: JoinOnRef, + ) -> PlanProperties { + // Calculate equivalence properties: + let eq_properties = join_equivalence_properties( + 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 = left.schema().fields.len(); + let output_partitioning = partitioned_join_output_partitioning( + join_type, + left.output_partitioning(), + right.output_partitioning(), + left_columns_len, + ); + + // Determine execution mode: + let mode = execution_mode_from_children([left, right]); + + PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -243,8 +266,8 @@ impl ExecutionPlan for SortMergeJoinExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -267,36 +290,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 506324852b21c..79b8c813d8603 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -43,14 +43,15 @@ 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::{ + execution_mode_from_children, expressions::PhysicalSortExpr, joins::StreamJoinPartitionMode, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, - Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::{ @@ -68,9 +69,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; @@ -174,8 +175,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 @@ -190,6 +189,8 @@ pub struct SymmetricHashJoinExec { pub(crate) right_sort_exprs: Option>, /// Partition Mode mode: StreamJoinPartitionMode, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl SymmetricHashJoinExec { @@ -230,14 +231,15 @@ 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::compute_properties(&left, &right, schema.clone(), *join_type, &on); Ok(SymmetricHashJoinExec { left, right, on, filter, join_type: *join_type, - schema: Arc::new(schema), random_state, metrics: ExecutionPlanMetricsSet::new(), column_indices, @@ -245,9 +247,45 @@ impl SymmetricHashJoinExec { left_sort_exprs, right_sort_exprs, mode, + cache, }) } + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + left: &Arc, + right: &Arc, + schema: SchemaRef, + join_type: JoinType, + join_on: JoinOnRef, + ) -> PlanProperties { + // Calculate equivalence properties: + let eq_properties = join_equivalence_properties( + left.equivalence_properties().clone(), + right.equivalence_properties().clone(), + &join_type, + schema, + &[false, false], + // Has alternating probe side + None, + join_on, + ); + + // Get output partitioning: + let left_columns_len = left.schema().fields.len(); + let output_partitioning = partitioned_join_output_partitioning( + join_type, + left.output_partitioning(), + right.output_partitioning(), + left_columns_len, + ); + + // Determine execution mode: + let mode = execution_mode_from_children([left, right]); + + PlanProperties::new(eq_properties, output_partitioning, mode) + } + /// left stream pub fn left(&self) -> &Arc { &self.left @@ -351,12 +389,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 properties(&self) -> &PlanProperties { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -389,34 +423,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()] } @@ -1326,11 +1332,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/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 37faae8737450..920e7b9d7a7cc 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/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index e6e3f83fd7e83..3dac0107d3ef5 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 a15fd470a98b5..b726b587dd1dd 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -29,18 +29,20 @@ 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::{plan_err, Result}; +use datafusion_common::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; 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}; @@ -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 { @@ -121,34 +119,12 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { fn as_any(&self) -> &dyn Any; /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef; - - /// Specifies how the output of this `ExecutionPlan` is split into - /// partitions. - fn output_partitioning(&self) -> 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 schema(&self) -> SchemaRef { + self.properties().schema().clone() } - /// 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]>; + /// Gets plan properties, such as output ordering(s), partitioning information etc. + fn properties(&self) -> &PlanProperties; /// Specifies the data distribution requirements for all the /// children for this `ExecutionPlan`, By default it's [[Distribution::UnspecifiedDistribution]] for each child, @@ -208,27 +184,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 { - EquivalenceProperties::new(self.schema()) - } - /// 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 @@ -446,6 +401,201 @@ 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; + + 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 [`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)` describing the ordering. A `None` return value + /// indicates no assumptions should be made on the output ordering. + /// + /// 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() + } + + /// 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 [`ExecutionPlan::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`. +#[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. + /// 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 + /// these streaming inputs. In this case, the execution mode will be pipeline + /// breaking, e.g. the operator requires unbounded memory to generate results. + PipelineBreaking, +} + +impl ExecutionMode { + /// Check whether the execution mode is unbounded or not. + pub fn is_unbounded(&self) -> bool { + matches!(self, ExecutionMode::Unbounded) + } + + /// 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) + } +} + +/// Conservatively "combines" execution modes of a given collection of operators. +fn execution_mode_from_children<'a>( + children: impl IntoIterator>, +) -> ExecutionMode { + let mut result = ExecutionMode::Bounded; + for mode in children.into_iter().map(|child| child.execution_mode()) { + match (mode, result) { + (ExecutionMode::PipelineBreaking, _) + | (_, ExecutionMode::PipelineBreaking) => { + // If any of the modes is `PipelineBreaking`, so is the result: + return ExecutionMode::PipelineBreaking; + } + (ExecutionMode::Unbounded, _) | (_, ExecutionMode::Unbounded) => { + // Unbounded mode eats up bounded mode: + result = ExecutionMode::Unbounded; + } + (ExecutionMode::Bounded, ExecutionMode::Bounded) => { + // When both modes are bounded, so is the result: + result = ExecutionMode::Bounded; + } + } + } + result +} + +/// Stores the plan properties used in query optimization. +/// +/// 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`]. + pub eq_properties: EquivalenceProperties, + /// Stores the output [`Partitioning`] of the [`ExecutionPlan`]. + pub partitioning: Partitioning, + /// Stores the [`ExecutionMode`] of the [`ExecutionPlan`]. + pub exec_mode: ExecutionMode, + /// Stores output ordering of the [`ExecutionPlan`]. A `None` value represents + /// no ordering. + output_ordering: Option, +} + +impl PlanProperties { + /// 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, + partitioning, + exec_mode, + output_ordering, + } + } + + /// Overwrite output partitioning with its new value. + pub fn with_partitioning(mut self, partitioning: Partitioning) -> Self { + self.partitioning = partitioning; + self + } + + /// 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. + pub fn with_eq_properties(mut self, eq_properties: EquivalenceProperties) -> Self { + // 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 + } + + 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() + } +} + /// 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 @@ -453,9 +603,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// 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::() { + if let Some(repartition) = plan.as_any().downcast_ref::() { !matches!( - repart.output_partitioning(), + repartition.properties().output_partitioning(), Partitioning::RoundRobinBatch(_) ) } else if let Some(coalesce) = plan.as_any().downcast_ref::() @@ -530,7 +680,7 @@ pub fn execute_stream( // merge into a single partition 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) } } @@ -595,17 +745,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 680aa23214f99..9fa15cbf64e2c 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, ExecutionPlanProperties, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; +use crate::{DisplayFormatType, Distribution, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -50,16 +50,19 @@ pub struct GlobalLimitExec { fetch: Option, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanProperties, } impl GlobalLimitExec { /// Create a new GlobalLimitExec pub fn new(input: Arc, skip: usize, fetch: Option) -> Self { + let cache = Self::compute_properties(&input); GlobalLimitExec { input, skip, fetch, metrics: ExecutionPlanMetricsSet::new(), + cache, } } @@ -77,6 +80,15 @@ impl GlobalLimitExec { pub fn fetch(&self) -> Option { self.fetch } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( + input.equivalence_properties().clone(), // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) + } } impl DisplayAs for GlobalLimitExec { @@ -104,8 +116,8 @@ impl ExecutionPlan for GlobalLimitExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() + fn properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -115,10 +127,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] @@ -128,14 +136,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>, @@ -147,10 +147,6 @@ impl ExecutionPlan for GlobalLimitExec { ))) } - fn unbounded_output(&self, _children: &[bool]) -> Result { - Ok(false) - } - fn execute( &self, partition: usize, @@ -271,15 +267,18 @@ pub struct LocalLimitExec { fetch: usize, /// Execution metrics metrics: ExecutionPlanMetricsSet, + cache: PlanProperties, } impl LocalLimitExec { /// Create a new LocalLimitExec partition pub fn new(input: Arc, fetch: usize) -> Self { + let cache = Self::compute_properties(&input); Self { input, fetch, metrics: ExecutionPlanMetricsSet::new(), + cache, } } @@ -292,6 +291,15 @@ impl LocalLimitExec { pub fn fetch(&self) -> usize { self.fetch } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( + input.equivalence_properties().clone(), // Equivalence Properties + input.output_partitioning().clone(), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) + } } impl DisplayAs for LocalLimitExec { @@ -314,39 +322,22 @@ impl ExecutionPlan for LocalLimitExec { self } - fn schema(&self) -> SchemaRef { - self.input.schema() + fn properties(&self) -> &PlanProperties { + &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>, @@ -419,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.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 86bd89e7ebac7..ca324a0f7d3b8 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, + PlanProperties, 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: PlanProperties, } 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 properties(&self) -> &PlanProperties { + &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,12 +153,14 @@ impl MemoryExec { projection: Option>, ) -> Result { let projected_schema = project_schema(&schema, projection.as_ref())?; + let cache = Self::compute_properties(projected_schema.clone(), &[], partitions); Ok(Self { partitions: partitions.to_vec(), schema, projected_schema, projection, sort_information: vec![], + cache, }) } @@ -203,12 +190,33 @@ 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; + + // We need to update equivalence properties when updating sort information. + let eq_properties = EquivalenceProperties::new_with_orderings( + self.schema(), + &self.sort_information, + ); + self.cache = self.cache.with_eq_properties(eq_properties); self } pub fn original_schema(&self) -> SchemaRef { self.schema.clone() } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + partitions: &[Vec], + ) -> PlanProperties { + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + PlanProperties::new( + eq_properties, // Equivalence Properties + Partitioning::UnknownPartitioning(partitions.len()), // Output Partitioning + ExecutionMode::Bounded, // Execution Mode + ) + } } /// Iterator over batches @@ -292,7 +300,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 +310,25 @@ 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!( + 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/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 3371148587d1f..37d209a3b4730 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, PlanProperties, SendableRecordBatchStream, + Statistics, +}; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::array::{ArrayRef, NullArray}; @@ -30,6 +32,7 @@ use arrow::record_batch::RecordBatch; use arrow_array::RecordBatchOptions; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use log::trace; @@ -40,20 +43,27 @@ pub struct PlaceholderRowExec { schema: SchemaRef, /// Number of partitions partitions: usize, + cache: PlanProperties, } impl PlaceholderRowExec { /// Create a new PlaceholderRowExec pub fn new(schema: SchemaRef) -> Self { + let partitions = 1; + let cache = Self::compute_properties(schema.clone(), partitions); PlaceholderRowExec { schema, - partitions: 1, + partitions, + 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(self.partitions); + self.cache = self.cache.with_partitioning(output_partitioning); self } @@ -79,6 +89,19 @@ impl PlaceholderRowExec { )?] }) } + + fn output_partitioning_helper(n_partitions: usize) -> Partitioning { + Partitioning::UnknownPartitioning(n_partitions) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + 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); + + PlanProperties::new(eq_properties, output_partitioning, ExecutionMode::Bounded) + } } impl DisplayAs for PlaceholderRowExec { @@ -101,23 +124,14 @@ impl ExecutionPlan for PlaceholderRowExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &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>, @@ -162,8 +176,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 28b9ff7b011d2..8fe82e7de3eb5 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, ExecutionPlanProperties, PlanProperties, 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,10 @@ 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 holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl ProjectionExec { @@ -70,6 +69,7 @@ impl ProjectionExec { input: Arc, ) -> Result { let input_schema = input.schema(); + let fields: Result> = expr .iter() .map(|(e, name)| { @@ -93,21 +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 mut input_eqs = input.equivalence_properties(); - - input_eqs.substitute_oeq_class(&projection_mapping)?; - - let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); - let output_ordering = project_eqs.oeq_class().output_ordering(); - + let cache = + Self::compute_properties(&input, &projection_mapping, schema.clone())?; Ok(Self { expr, schema, input, - output_ordering, - projection_mapping, metrics: ExecutionPlanMetricsSet::new(), + cache, }) } @@ -120,6 +113,43 @@ impl ProjectionExec { pub fn input(&self) -> &Arc { &self.input } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + 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(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(); + let output_partitioning = if let Partitioning::Hash(exprs, part) = input_partition + { + let normalized_exprs = exprs + .iter() + .map(|expr| { + input_eq_properties + .project_expr(expr, projection_mapping) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) + }) + .collect(); + Partitioning::Hash(normalized_exprs, *part) + } else { + input_partition.clone() + }; + + Ok(PlanProperties::new( + eq_properties, + output_partitioning, + input.execution_mode(), + )) + } } impl DisplayAs for ProjectionExec { @@ -155,61 +185,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 properties(&self) -> &PlanProperties { + &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 { - let mut equi_properties = self.input.equivalence_properties(); - equi_properties - .substitute_oeq_class(&self.projection_mapping) - .unwrap(); - equi_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 614ab990ac49a..9786b1cbf6fdb 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -21,24 +21,21 @@ 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}, + metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, work_table::{WorkTable, WorkTableExec}, - SendableRecordBatchStream, Statistics, + PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::{DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan}; use arrow::datatypes::SchemaRef; 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 futures::{ready, Stream, StreamExt}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan}; +use futures::{ready, Stream, StreamExt}; /// Recursive query execution plan. /// @@ -69,6 +66,8 @@ pub struct RecursiveQueryExec { is_distinct: bool, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl RecursiveQueryExec { @@ -83,6 +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::compute_properties(static_term.schema()); Ok(RecursiveQueryExec { name, static_term, @@ -90,8 +90,20 @@ impl RecursiveQueryExec { is_distinct, work_table, metrics: ExecutionPlanMetricsSet::new(), + cache, }) } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } } impl ExecutionPlan for RecursiveQueryExec { @@ -99,21 +111,14 @@ impl ExecutionPlan for RecursiveQueryExec { self } - fn schema(&self) -> SchemaRef { - self.static_term.schema() + fn properties(&self) -> &PlanProperties { + &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 +136,17 @@ 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(), - })) + 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 a66a929796aba..fe93ea1315063 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -24,34 +24,34 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::{any::Any, vec}; +use super::common::SharedMemoryReservation; +use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use super::{ + DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream, +}; +use crate::common::{transpose, SpawnedTask}; +use crate::hash_utils::create_hashes; +use crate::metrics::BaselineMetrics; +use crate::repartition::distributor_channels::{ + channels, partition_aware_channels, DistributionReceiver, DistributionSender, +}; +use crate::sorts::streaming_merge; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; + 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::{EquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; + use futures::stream::Stream; use futures::{FutureExt, StreamExt}; use hashbrown::HashMap; use log::trace; use parking_lot::Mutex; -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 crate::common::{transpose, SpawnedTask}; -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 super::common::SharedMemoryReservation; -use super::expressions::PhysicalSortExpr; -use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream}; - -use self::distributor_channels::{DistributionReceiver, DistributionSender}; - mod distributor_channels; type MaybeBatch = Option>; @@ -294,19 +294,17 @@ 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: PlanProperties, } #[derive(Debug, Clone)] @@ -412,9 +410,8 @@ impl ExecutionPlan for RepartitionExec { self } - /// Get the schema for this execution plan - fn schema(&self) -> SchemaRef { - self.input.schema() + fn properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -433,45 +430,12 @@ 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] - } else { - // We preserve ordering when input partitioning is 1 - vec![self.input().output_partitioning().partition_count() <= 1] - } - } - - 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 + Self::maintains_input_order_helper(self.input(), self.preserve_order) } fn execute( @@ -631,6 +595,9 @@ impl RepartitionExec { input: Arc, partitioning: Partitioning, ) -> Result { + let preserve_order = false; + let cache = + Self::compute_properties(&input, partitioning.clone(), preserve_order); Ok(RepartitionExec { input, partitioning, @@ -639,10 +606,48 @@ impl RepartitionExec { abort_helper: Arc::new(Vec::new()), })), metrics: ExecutionPlanMetricsSet::new(), - preserve_order: false, + preserve_order, + cache, }) } + fn maintains_input_order_helper( + input: &Arc, + preserve_order: bool, + ) -> Vec { + // 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( + input: &Arc, + preserve_order: bool, + ) -> EquivalenceProperties { + // Equivalence Properties + let mut eq_properties = input.equivalence_properties().clone(); + // If the ordering is lost, reset the ordering equivalence class: + if !Self::maintains_input_order_helper(input, preserve_order)[0] { + eq_properties.clear_orderings(); + } + eq_properties + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + partitioning: Partitioning, + preserve_order: bool, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = Self::eq_properties_helper(input, preserve_order); + + PlanProperties::new( + eq_properties, // Equivalence Properties + partitioning, // Output Partitioning + input.execution_mode(), // Execution Mode + ) + } + /// Specify if this reparititoning operation should preserve the order of /// rows from its input when producing output. Preserving order is more /// expensive at runtime, so should only be set if the output of this @@ -657,6 +662,8 @@ impl RepartitionExec { // if there is only one input partition, merging is not required // to maintain order self.input.output_partitioning().partition_count() > 1; + let eq_properties = Self::eq_properties_helper(&self.input, self.preserve_order); + self.cache = self.cache.with_eq_properties(eq_properties); self } @@ -919,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, @@ -941,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/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 563a82f3ea7b3..500df6153fdba 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -57,24 +57,24 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +use crate::expressions::PhysicalSortExpr; +use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::sorts::sort::sort_batch; +use crate::{ + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, +}; + 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 datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::LexOrdering; -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, -}; +use futures::{ready, Stream, StreamExt}; +use log::trace; /// Partial Sort execution plan. #[derive(Debug, Clone)] @@ -93,6 +93,8 @@ pub struct PartialSortExec { preserve_partitioning: bool, /// Fetch highest/lowest n results fetch: Option, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl PartialSortExec { @@ -103,13 +105,16 @@ impl PartialSortExec { common_prefix_length: usize, ) -> Self { assert!(common_prefix_length > 0); + let preserve_partitioning = false; + let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning); Self { input, expr, common_prefix_length, metrics_set: ExecutionPlanMetricsSet::new(), - preserve_partitioning: false, + preserve_partitioning, fetch: None, + cache, } } @@ -127,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 } @@ -156,6 +167,41 @@ impl PartialSortExec { pub fn fetch(&self) -> Option { self.fetch } + + fn output_partitioning_helper( + input: &Arc, + preserve_partitioning: bool, + ) -> Partitioning { + // Get output partitioning: + if preserve_partitioning { + input.output_partitioning().clone() + } else { + Partitioning::UnknownPartitioning(1) + } + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + sort_exprs: LexOrdering, + preserve_partitioning: bool, + ) -> PlanProperties { + // Calculate equivalence properties; i.e. reset the ordering equivalence + // class with the new ordering: + let eq_properties = input + .equivalence_properties() + .clone() + .with_reorder(sort_exprs); + + // Get output partitioning: + let output_partitioning = + Self::output_partitioning_helper(input, preserve_partitioning); + + // Determine execution mode: + let mode = input.execution_mode(); + + PlanProperties::new(eq_properties, output_partitioning, mode) + } } impl DisplayAs for PartialSortExec { @@ -184,28 +230,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 properties(&self) -> &PlanProperties { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -220,13 +246,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 84bf3ec415ef1..f469586632527 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -36,22 +36,23 @@ 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, ExecutionPlanProperties, Partitioning, PlanProperties, + 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 datafusion_physical_expr::LexOrdering; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; @@ -675,6 +676,8 @@ pub struct SortExec { preserve_partitioning: bool, /// Fetch highest/lowest n results fetch: Option, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl SortExec { @@ -691,12 +694,15 @@ impl SortExec { /// Create a new sort execution plan that produces a single, /// sorted output partition. pub fn new(expr: Vec, input: Arc) -> Self { + let preserve_partitioning = false; + let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning); Self { expr, input, metrics_set: ExecutionPlanMetricsSet::new(), - preserve_partitioning: false, + preserve_partitioning, fetch: None, + cache, } } @@ -731,6 +737,12 @@ 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.cache = self + .cache + .with_partitioning(Self::output_partitioning_helper( + &self.input, + self.preserve_partitioning, + )); self } @@ -760,6 +772,46 @@ impl SortExec { pub fn fetch(&self) -> Option { self.fetch } + + fn output_partitioning_helper( + input: &Arc, + preserve_partitioning: bool, + ) -> Partitioning { + // Get output partitioning: + if preserve_partitioning { + input.output_partitioning().clone() + } else { + Partitioning::UnknownPartitioning(1) + } + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + sort_exprs: LexOrdering, + preserve_partitioning: bool, + ) -> PlanProperties { + // Calculate equivalence properties; i.e. reset the ordering equivalence + // class with the new ordering: + let eq_properties = input + .equivalence_properties() + .clone() + .with_reorder(sort_exprs); + + // Get output partitioning: + let output_partitioning = + Self::output_partitioning_helper(input, preserve_partitioning); + + // Determine execution mode: + let mode = match input.execution_mode() { + ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { + ExecutionMode::PipelineBreaking + } + ExecutionMode::Bounded => ExecutionMode::Bounded, + }; + + PlanProperties::new(eq_properties, output_partitioning, mode) + } } impl DisplayAs for SortExec { @@ -787,28 +839,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 properties(&self) -> &PlanProperties { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -829,17 +861,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 81a26cd2188de..556615f64de62 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -25,15 +25,14 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::streaming_merge; use crate::{ - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, + Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, }; -use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, 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,16 +73,20 @@ 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: PlanProperties, } impl SortPreservingMergeExec { /// Create a new sort execution plan pub fn new(expr: Vec, input: Arc) -> Self { + let cache = Self::compute_properties(&input); Self { input, expr, metrics: ExecutionPlanMetricsSet::new(), fetch: None, + cache, } } /// Sets the number of rows to fetch @@ -106,6 +109,15 @@ impl SortPreservingMergeExec { pub fn fetch(&self) -> Option { self.fetch } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( + input.equivalence_properties().clone(), // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + input.execution_mode(), // Execution Mode + ) + } } impl DisplayAs for SortPreservingMergeExec { @@ -137,20 +149,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 properties(&self) -> &PlanProperties { + &self.cache } fn required_input_distribution(&self) -> Vec { @@ -165,18 +165,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/stream.rs b/datafusion/physical-plan/src/stream.rs index 970194550045e..99d9367740be4 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)])) } @@ -555,7 +554,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/streaming.rs b/datafusion/physical-plan/src/streaming.rs index bca37ed942d01..7b062ab8741fd 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, PlanProperties}; use crate::display::{display_orderings, 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, 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: PlanProperties, } impl StreamingTableExec { @@ -84,13 +85,21 @@ impl StreamingTableExec { Some(p) => Arc::new(schema.project(p)?), None => schema, }; - + let projected_output_ordering = + projected_output_ordering.into_iter().collect::>(); + let cache = Self::compute_properties( + 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, }) } @@ -117,6 +126,29 @@ impl StreamingTableExec { pub fn is_infinite(&self) -> bool { self.infinite } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + orderings: &[LexOrdering], + partitions: &[Arc], + is_infinite: bool, + ) -> PlanProperties { + // Calculate equivalence properties: + let eq_properties = EquivalenceProperties::new_with_orderings(schema, orderings); + + // Get output partitioning: + let output_partitioning = Partitioning::UnknownPartitioning(partitions.len()); + + // Determine execution mode: + let mode = if is_infinite { + ExecutionMode::Unbounded + } else { + ExecutionMode::Bounded + }; + + PlanProperties::new(eq_properties, output_partitioning, mode) + } } impl std::fmt::Debug for StreamingTableExec { @@ -163,29 +195,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 properties(&self) -> &PlanProperties { + &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 5a8ef2db77c28..23df3753e8173 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -26,15 +26,15 @@ use std::{ use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::{ - common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + common, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, + PlanProperties, 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 datafusion_physical_expr::EquivalenceProperties; use futures::Stream; use tokio::sync::Barrier; @@ -121,6 +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: PlanProperties, } impl MockExec { @@ -132,10 +133,12 @@ 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::compute_properties(schema.clone()); Self { data, schema, use_task: true, + cache, } } @@ -146,6 +149,17 @@ impl MockExec { self.use_task = use_task; self } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for MockExec { @@ -167,20 +181,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { - unimplemented!() + vec![] } fn with_new_children( @@ -276,6 +282,7 @@ pub struct BarrierExec { /// all streams wait on this barrier to produce barrier: Arc, + cache: PlanProperties, } impl BarrierExec { @@ -283,10 +290,12 @@ 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::compute_properties(schema.clone(), &data); Self { data, schema, barrier, + cache, } } @@ -296,6 +305,19 @@ impl BarrierExec { self.barrier.wait().await; 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 compute_properties( + schema: SchemaRef, + data: &[Vec], + ) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(data.len()), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for BarrierExec { @@ -317,16 +339,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -383,7 +397,7 @@ impl ExecutionPlan for BarrierExec { /// A mock execution plan that errors on a call to execute #[derive(Debug)] pub struct ErrorExec { - schema: SchemaRef, + cache: PlanProperties, } impl Default for ErrorExec { @@ -399,7 +413,19 @@ impl ErrorExec { DataType::Int64, true, )])); - Self { schema } + 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 compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) } } @@ -422,16 +448,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -460,6 +478,7 @@ impl ExecutionPlan for ErrorExec { pub struct StatisticsExec { stats: Statistics, schema: Arc, + cache: PlanProperties, } impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { @@ -468,11 +487,24 @@ impl StatisticsExec { .column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); + let cache = Self::compute_properties(Arc::new(schema.clone())); Self { stats, schema: Arc::new(schema), + cache, } } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(2), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for StatisticsExec { @@ -499,16 +531,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 properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -543,20 +567,19 @@ 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: 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::compute_properties(schema.clone(), n_partitions); Self { schema, - n_partitions, refs: Default::default(), + cache, } } @@ -568,6 +591,17 @@ impl BlockingExec { pub fn refs(&self) -> Weak<()> { Arc::downgrade(&self.refs) } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef, n_partitions: usize) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(n_partitions), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for BlockingExec { @@ -589,8 +623,8 @@ impl ExecutionPlan for BlockingExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) + fn properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -598,14 +632,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,15 +707,19 @@ 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: PlanProperties, } 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 batches_until_panics = vec![0; n_partitions]; + let cache = Self::compute_properties(schema.clone(), &batches_until_panics); Self { schema, - batches_until_panics: vec![0; n_partitions], + batches_until_panics, + cache, } } @@ -698,6 +728,21 @@ impl PanicExec { self.batches_until_panics[partition] = count; self } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + batches_until_panics: &[usize], + ) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + let num_partitions = batches_until_panics.len(); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(num_partitions), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for PanicExec { @@ -719,8 +764,8 @@ impl ExecutionPlan for PanicExec { self } - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) + fn properties(&self) -> &PlanProperties { + &self.cache } fn children(&self) -> Vec> { @@ -728,15 +773,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 62a6d5c0f8776..7eaac74a54499 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -27,12 +27,12 @@ use std::task::{Context, Poll}; use std::{any::Any, sync::Arc}; use super::{ - expressions::PhysicalSortExpr, + execution_mode_from_children, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, + ExecutionPlanProperties, Partitioning, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; -use crate::common::get_meet_of_orderings; use crate::metrics::BaselineMetrics; use crate::stream::ObservedStream; @@ -91,19 +91,19 @@ pub struct UnionExec { inputs: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Schema of Union - schema: SchemaRef, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl UnionExec { /// Create a new UnionExec pub fn new(inputs: Vec>) -> Self { let schema = union_schema(&inputs); - + let cache = Self::compute_properties(&inputs, schema); UnionExec { inputs, metrics: ExecutionPlanMetricsSet::new(), - schema, + cache, } } @@ -111,6 +111,61 @@ impl UnionExec { pub fn inputs(&self) -> &Vec> { &self.inputs } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + inputs: &[Arc], + schema: SchemaRef, + ) -> PlanProperties { + // 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 = inputs + .iter() + .map(|child| child.equivalence_properties()) + .collect::>(); + 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() + .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); + + // Calculate output partitioning; i.e. sum output partitions of the 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 = execution_mode_from_children(inputs.iter()); + + PlanProperties::new(eq_properties, output_partitioning, mode) + } } 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 properties(&self) -> &PlanProperties { + &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). @@ -176,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.properties().output_ordering() { self.inputs() .iter() .map(|child| { @@ -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,25 +317,23 @@ pub struct InterleaveExec { inputs: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Schema of Interleave - schema: SchemaRef, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } 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 = Self::compute_properties(&inputs); Ok(InterleaveExec { inputs, metrics: ExecutionPlanMetricsSet::new(), - schema, + cache, }) } @@ -354,6 +341,18 @@ impl InterleaveExec { pub fn inputs(&self) -> &Vec> { &self.inputs } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + 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 = execution_mode_from_children(inputs.iter()); + + PlanProperties::new(eq_properties, output_partitioning, mode) + } } impl DisplayAs for InterleaveExec { @@ -376,32 +375,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 properties(&self) -> &PlanProperties { + &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 +466,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 +595,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 { @@ -654,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); @@ -825,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: {:?}", diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index d5453f0924a38..886b718e6efe1 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -19,31 +19,30 @@ //! 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::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use super::{DisplayAs, ExecutionPlanProperties, PlanProperties}; use crate::{ - expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PhysicalExpr, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, + expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, PhysicalExpr, + RecordBatchStream, SendableRecordBatchStream, }; 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, Result, UnnestOptions}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; 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 +59,8 @@ pub struct UnnestExec { options: UnnestOptions, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl UnnestExec { @@ -70,14 +71,30 @@ impl UnnestExec { schema: SchemaRef, options: UnnestOptions, ) -> Self { + let cache = Self::compute_properties(&input, schema.clone()); UnnestExec { input, schema, column, options, metrics: Default::default(), + cache, } } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + schema: SchemaRef, + ) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + input.output_partitioning().clone(), + input.execution_mode(), + ) + } } impl DisplayAs for UnnestExec { @@ -99,21 +116,14 @@ impl ExecutionPlan for UnnestExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &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 +140,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 9c1ce93b2a081..8868a59008b71 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, PlanProperties, SendableRecordBatchStream, + Statistics, +}; use crate::{ memory::MemoryStream, ColumnarValue, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, @@ -31,6 +33,7 @@ use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::{internal_err, plan_err, Result, ScalarValue}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; /// Execution plan for values list based relation (produces constant rows) #[derive(Debug)] @@ -39,6 +42,8 @@ pub struct ValuesExec { schema: SchemaRef, /// The data data: Vec, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl ValuesExec { @@ -85,7 +90,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,9 +114,11 @@ impl ValuesExec { } } + let cache = Self::compute_properties(schema.clone()); Ok(ValuesExec { schema, data: batches, + cache, }) } @@ -119,6 +126,17 @@ impl ValuesExec { pub fn data(&self) -> Vec { self.data.clone() } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for ValuesExec { @@ -141,30 +159,20 @@ impl ExecutionPlan for ValuesExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &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(), - })) + 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 9d247d689c076..c99ec599596d9 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, Partitioning, RecordBatchStream, SendableRecordBatchStream, - Statistics, WindowExpr, + ExecutionPlanProperties, InputOrderMode, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, Statistics, WindowExpr, }; use arrow::{ @@ -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,8 @@ 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: PlanProperties, } impl BoundedWindowAggExec { @@ -121,6 +121,7 @@ impl BoundedWindowAggExec { vec![] } }; + let cache = Self::compute_properties(&input, &schema, &window_expr); Ok(Self { input, window_expr, @@ -129,6 +130,7 @@ impl BoundedWindowAggExec { metrics: ExecutionPlanMetricsSet::new(), input_order_mode, ordered_partition_by_indices, + cache, }) } @@ -179,6 +181,28 @@ impl BoundedWindowAggExec { } }) } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + schema: &SchemaRef, + window_expr: &[Arc], + ) -> PlanProperties { + // Calculate equivalence properties: + 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 = input.output_partitioning().clone(); + + // Construct properties cache + PlanProperties::new( + eq_properties, // Equivalence Properties + output_partitioning, // Output Partitioning + input.execution_mode(), // Execution Mode + ) + } } impl DisplayAs for BoundedWindowAggExec { @@ -216,30 +240,14 @@ impl ExecutionPlan for BoundedWindowAggExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &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] } @@ -1114,21 +1117,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 bf6ed925356c1..c19694aef8b79 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, ExecutionPlanProperties, InputOrderMode, PhysicalExpr, }; use arrow::datatypes::Schema; @@ -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 #[allow(clippy::too_many_arguments)] @@ -377,8 +376,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) = @@ -420,7 +419,7 @@ pub fn get_best_fitting_window( } else { return Ok(None); }; - let is_unbounded = unbounded_output(input); + 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: @@ -482,7 +481,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 e80102812ebd8..e300eee49d316 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -30,27 +30,23 @@ 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, ExecutionPlanProperties, PhysicalExpr, PlanProperties, + RecordBatchStream, 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, plan_err, Result}; +use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; +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,8 @@ 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: PlanProperties, } impl WindowAggExec { @@ -82,6 +80,7 @@ impl WindowAggExec { let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); + let cache = Self::compute_properties(schema.clone(), &input, &window_expr); Ok(Self { input, window_expr, @@ -89,6 +88,7 @@ impl WindowAggExec { partition_keys, metrics: ExecutionPlanMetricsSet::new(), ordered_partition_by_indices, + cache, }) } @@ -115,6 +115,32 @@ impl WindowAggExec { &self.ordered_partition_by_indices, ) } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + input: &Arc, + window_expr: &[Arc], + ) -> PlanProperties { + // Calculate equivalence properties: + 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 = input.output_partitioning().clone(); + + // Determine execution mode: + let mode = match input.execution_mode() { + ExecutionMode::Bounded => ExecutionMode::Bounded, + ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { + ExecutionMode::PipelineBreaking + } + }; + + // Construct properties cache: + PlanProperties::new(eq_properties, output_partitioning, mode) + } } impl DisplayAs for WindowAggExec { @@ -151,39 +177,14 @@ impl ExecutionPlan for WindowAggExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &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 +211,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 0f934a76a60f5..0e138b8f74a76 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -20,21 +20,18 @@ 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 crate::memory::MemoryStream; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan}; - -use super::expressions::PhysicalSortExpr; - use super::{ metrics::{ExecutionPlanMetricsSet, MetricsSet}, SendableRecordBatchStream, Statistics, }; +use crate::memory::MemoryStream; +use crate::{DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, PlanProperties}; + +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, Result}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; /// The name is from PostgreSQL's terminology. /// See @@ -85,16 +82,20 @@ pub struct WorkTableExec { work_table: Arc, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, } impl WorkTableExec { /// Create a new execution plan for a worktable exec. pub fn new(name: String, schema: SchemaRef) -> Self { + let cache = Self::compute_properties(schema.clone()); Self { name, schema, metrics: ExecutionPlanMetricsSet::new(), work_table: Arc::new(WorkTable::new()), + cache, } } @@ -104,8 +105,20 @@ impl WorkTableExec { schema: self.schema.clone(), metrics: ExecutionPlanMetricsSet::new(), work_table, + cache: self.cache.clone(), } } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let eq_properties = EquivalenceProperties::new(schema); + + PlanProperties::new( + eq_properties, + Partitioning::UnknownPartitioning(1), + ExecutionMode::Bounded, + ) + } } impl DisplayAs for WorkTableExec { @@ -127,18 +140,14 @@ impl ExecutionPlan for WorkTableExec { self } - fn schema(&self) -> SchemaRef { - self.schema.clone() + fn properties(&self) -> &PlanProperties { + &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 +156,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 8d6b314747bb6..a541d03701844 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3580,7 +3580,7 @@ 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_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]]