diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 442e6e4009f6..06db092d6fc8 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -37,4 +37,4 @@ If there are user-facing changes then we may require documentation to be updated \ No newline at end of file +--> diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 900aad95747e..a9bd7d0e27bb 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -106,7 +106,7 @@ const ARROW_MAGIC: [u8; 6] = [b'A', b'R', b'R', b'O', b'W', b'1']; const CONTINUATION_MARKER: [u8; 4] = [0xff; 4]; /// Custom implementation of inferring schema. Should eventually be moved upstream to arrow-rs. -/// See https://github.com/apache/arrow-rs/issues/5021 +/// See async fn infer_schema_from_file_stream( mut stream: BoxStream<'static, object_store::Result>, ) -> Result { diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index e00e8aea0a04..30b55db28491 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -32,10 +32,7 @@ use crate::physical_plan::{ use arrow_schema::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, - PhysicalSortExpr, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; @@ -106,8 +103,8 @@ impl ExecutionPlan for ArrowExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 237772eb8360..b97f162fd2f5 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -31,9 +31,7 @@ use crate::physical_plan::{ use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] @@ -101,8 +99,8 @@ impl ExecutionPlan for AvroExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 82163da64af8..75aa343ffbfc 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -41,9 +41,7 @@ use crate::physical_plan::{ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; use datafusion_common::config::ConfigOptions; @@ -159,8 +157,8 @@ impl ExecutionPlan for CsvExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 1ba8e47a523c..73dcb32ac81f 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -40,9 +40,7 @@ use crate::physical_plan::{ use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; use futures::{ready, stream, StreamExt, TryStreamExt}; @@ -122,8 +120,8 @@ impl ExecutionPlan for NdJsonExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index f6e999f60249..960b2ec7337d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -45,8 +45,7 @@ use crate::{ use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, + EquivalenceProperties, LexOrdering, PhysicalExpr, PhysicalSortExpr, }; use bytes::Bytes; @@ -315,8 +314,8 @@ impl ExecutionPlan for ParquetExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 6de39db7d52a..ee6e11bd271a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -49,12 +49,11 @@ use crate::physical_plan::{ use arrow::compute::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; -use datafusion_physical_expr::equivalence::EquivalenceProperties; use datafusion_physical_expr::expressions::{Column, NoOp}; -use datafusion_physical_expr::utils::{ - map_columns_before_projection, ordering_satisfy_requirement_concrete, +use datafusion_physical_expr::utils::map_columns_before_projection; +use datafusion_physical_expr::{ + physical_exprs_equal, EquivalenceProperties, PhysicalExpr, }; -use datafusion_physical_expr::{expr_list_eq_strict_order, PhysicalExpr}; use datafusion_physical_plan::unbounded_output; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; @@ -498,7 +497,7 @@ fn reorder_aggregate_keys( if parent_required.len() != output_exprs.len() || !agg_exec.group_by().null_expr().is_empty() - || expr_list_eq_strict_order(&output_exprs, parent_required) + || physical_exprs_equal(&output_exprs, parent_required) { Ok(PlanWithKeyRequirements::new(agg_plan)) } else { @@ -564,13 +563,11 @@ fn reorder_aggregate_keys( Arc::new(Column::new( name, agg_schema.index_of(name).unwrap(), - )) - as Arc, + )) as _, name.to_owned(), ) }) .collect::>(); - let agg_schema = new_final_agg.schema(); let agg_fields = agg_schema.fields(); for (idx, field) in agg_fields.iter().enumerate().skip(output_columns.len()) @@ -706,10 +703,9 @@ pub(crate) fn reorder_join_keys_to_inputs( ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); - let mut new_sort_options = vec![]; - for idx in 0..sort_options.len() { - new_sort_options.push(sort_options[new_positions[idx]]) - } + let new_sort_options = (0..sort_options.len()) + .map(|idx| sort_options[new_positions[idx]]) + .collect(); return Ok(Arc::new(SortMergeJoinExec::try_new( left.clone(), right.clone(), @@ -757,39 +753,40 @@ fn try_reorder( expected: &[Arc], equivalence_properties: &EquivalenceProperties, ) -> Option<(JoinKeyPairs, Vec)> { + 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; } - if expr_list_eq_strict_order(expected, &join_keys.left_keys) - || expr_list_eq_strict_order(expected, &join_keys.right_keys) + if physical_exprs_equal(expected, &join_keys.left_keys) + || physical_exprs_equal(expected, &join_keys.right_keys) { return Some((join_keys, vec![])); - } else if !equivalence_properties.classes().is_empty() { + } else if !equivalence_properties.eq_group().is_empty() { normalized_expected = expected .iter() - .map(|e| equivalence_properties.normalize_expr(e.clone())) + .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| equivalence_properties.normalize_expr(e.clone())) + .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| equivalence_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); assert_eq!(join_keys.right_keys.len(), normalized_right_keys.len()); - if expr_list_eq_strict_order(&normalized_expected, &normalized_left_keys) - || expr_list_eq_strict_order(&normalized_expected, &normalized_right_keys) + if physical_exprs_equal(&normalized_expected, &normalized_left_keys) + || physical_exprs_equal(&normalized_expected, &normalized_right_keys) { return Some((join_keys, vec![])); } @@ -870,7 +867,7 @@ fn new_join_conditions( r_key.as_any().downcast_ref::().unwrap().clone(), ) }) - .collect::>() + .collect() } /// Updates `dist_onward` such that, to keep track of @@ -935,9 +932,9 @@ fn add_roundrobin_on_top( let should_preserve_ordering = input.output_ordering().is_some(); let partitioning = Partitioning::RoundRobinBatch(n_target); - let repartition = RepartitionExec::try_new(input, partitioning)? - .with_preserve_order(should_preserve_ordering); - let new_plan = Arc::new(repartition) as Arc; + let repartition = RepartitionExec::try_new(input, partitioning)?; + let new_plan = Arc::new(repartition.with_preserve_order(should_preserve_ordering)) + as Arc; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1011,9 +1008,9 @@ fn add_hash_on_top( input }; let partitioning = Partitioning::Hash(hash_exprs, n_target); - let repartition = RepartitionExec::try_new(new_plan, partitioning)? - .with_preserve_order(should_preserve_ordering); - new_plan = Arc::new(repartition) as _; + let repartition = RepartitionExec::try_new(new_plan, partitioning)?; + new_plan = + Arc::new(repartition.with_preserve_order(should_preserve_ordering)) as _; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1302,16 +1299,12 @@ fn ensure_distribution( // There is an ordering requirement of the operator: if let Some(required_input_ordering) = required_input_ordering { - let existing_ordering = child.output_ordering().unwrap_or(&[]); // Either: // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or // - using order preserving variant is not desirable. - let ordering_satisfied = ordering_satisfy_requirement_concrete( - existing_ordering, - required_input_ordering, - || child.equivalence_properties(), - || child.ordering_equivalence_properties(), - ); + let ordering_satisfied = child + .equivalence_properties() + .ordering_satisfy_requirement(required_input_ordering); if !ordering_satisfied || !order_preserving_variants_desirable { replace_order_preserving_variants(&mut child, dist_onward)?; // If ordering requirements were satisfied before repartitioning, @@ -3763,14 +3756,14 @@ mod tests { fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + expr: col("a", &schema).unwrap(), options: SortOptions::default(), }]; let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "SortExec: expr=[c@2 ASC]", + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -3780,7 +3773,7 @@ mod tests { assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", "FilterExec: c@2 = 0", // Expect repartition on the input of the filter (as it can benefit from additional parallelism) @@ -4357,29 +4350,54 @@ mod tests { fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + expr: col("a", &schema).unwrap(), options: SortOptions::default(), }]; let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "SortExec: expr=[c@2 ASC]", + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_optimized!(expected, physical_plan.clone(), true); let expected = &[ - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", + "CoalescePartitionsExec", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected, physical_plan, false); + + Ok(()) + } + + #[test] + fn no_need_for_sort_after_filter() -> Result<()> { + let schema = schema(); + let sort_key = vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]; + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let expected = &[ + // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. "CoalescePartitionsExec", + // Since after this stage c is constant. c@2 ASC ordering is already satisfied. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; + + assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); Ok(()) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 822a224d236a..4779ced44f1a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -51,18 +51,16 @@ use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::windows::{ - get_best_fitting_window, BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, + get_best_fitting_window, BoundedWindowAggExec, WindowAggExec, }; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; -use datafusion_physical_expr::utils::{ - ordering_satisfy, ordering_satisfy_requirement_concrete, -}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::windows::PartitionSearchMode; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -451,13 +449,11 @@ fn ensure_sorting( { let physical_ordering = child.output_ordering(); match (required_ordering, physical_ordering) { - (Some(required_ordering), Some(physical_ordering)) => { - if !ordering_satisfy_requirement_concrete( - physical_ordering, - &required_ordering, - || child.equivalence_properties(), - || child.ordering_equivalence_properties(), - ) { + (Some(required_ordering), Some(_)) => { + if !child + .equivalence_properties() + .ordering_satisfy_requirement(&required_ordering) + { // Make sure we preserve the ordering requirements: update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; add_sort_above(child, &required_ordering, None); @@ -516,13 +512,12 @@ fn analyze_immediate_sort_removal( ) -> Option { if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_input = sort_exec.input().clone(); + // If this sort is unnecessary, we should remove it: - if ordering_satisfy( - sort_input.output_ordering(), - sort_exec.output_ordering(), - || sort_input.equivalence_properties(), - || sort_input.ordering_equivalence_properties(), - ) { + if sort_input + .equivalence_properties() + .ordering_satisfy(sort_exec.output_ordering().unwrap_or(&[])) + { // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: return Some( diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index f5eacd5ee60c..d9cdc292dd56 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -33,7 +33,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{ - Distribution, LexOrderingReq, PhysicalSortExpr, PhysicalSortRequirement, + Distribution, LexRequirement, PhysicalSortExpr, PhysicalSortRequirement, }; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -90,14 +90,14 @@ enum RuleMode { #[derive(Debug)] struct OutputRequirementExec { input: Arc, - order_requirement: Option, + order_requirement: Option, dist_requirement: Distribution, } impl OutputRequirementExec { fn new( input: Arc, - requirements: Option, + requirements: Option, dist_requirement: Distribution, ) -> Self { Self { 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 fb75c083a70a..0c2f21d11acd 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -31,7 +31,6 @@ use super::utils::is_repartition; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_physical_expr::utils::ordering_satisfy; use datafusion_physical_plan::unbounded_output; /// For a given `plan`, this object carries the information one needs from its @@ -258,12 +257,10 @@ pub(crate) fn replace_with_order_preserving_variants( is_spm_better || use_order_preserving_variant, )?; // If this sort is unnecessary, we should remove it and update the plan: - if ordering_satisfy( - updated_sort_input.output_ordering(), - plan.output_ordering(), - || updated_sort_input.equivalence_properties(), - || updated_sort_input.ordering_equivalence_properties(), - ) { + if updated_sort_input + .equivalence_properties() + .ordering_satisfy(plan.output_ordering().unwrap_or(&[])) + { return Ok(Transformed::Yes(OrderPreservationContext { plan: updated_sort_input, ordering_onwards: vec![None], @@ -278,21 +275,21 @@ pub(crate) fn replace_with_order_preserving_variants( mod tests { use super::*; - use crate::prelude::SessionConfig; - use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::filter::FilterExec; use crate::physical_plan::joins::{HashJoinExec, PartitionMode}; 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::{displayable, Partitioning}; + use crate::prelude::SessionConfig; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::tree_node::TreeNode; use datafusion_common::{Result, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; @@ -300,9 +297,6 @@ mod tests { use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts the plan /// against the original and expected plans. /// diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index a99399592f15..b9502d92ac12 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -29,15 +29,12 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{ - plan_datafusion_err, plan_err, DataFusionError, JoinSide, Result, -}; +use datafusion_common::{plan_err, DataFusionError, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::utils::{ - ordering_satisfy, ordering_satisfy_requirement, requirements_compatible, +use datafusion_physical_expr::{ + LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; -use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use itertools::izip; @@ -127,29 +124,27 @@ pub(crate) fn pushdown_sorts( requirements: SortPushDown, ) -> Result> { let plan = &requirements.plan; - let parent_required = requirements.required_ordering.as_deref(); - const ERR_MSG: &str = "Expects parent requirement to contain something"; - let err = || plan_datafusion_err!("{}", ERR_MSG); + let parent_required = requirements.required_ordering.as_deref().unwrap_or(&[]); if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let mut new_plan = plan.clone(); - if !ordering_satisfy_requirement( - plan.output_ordering(), - parent_required, - || plan.equivalence_properties(), - || plan.ordering_equivalence_properties(), - ) { + let new_plan = if !plan + .equivalence_properties() + .ordering_satisfy_requirement(parent_required) + { // If the current plan is a SortExec, modify it to satisfy parent requirements: - let parent_required_expr = parent_required.ok_or_else(err)?; - new_plan = sort_exec.input().clone(); - add_sort_above(&mut new_plan, parent_required_expr, sort_exec.fetch()); + let mut new_plan = sort_exec.input().clone(); + add_sort_above(&mut new_plan, parent_required, sort_exec.fetch()); + new_plan + } else { + requirements.plan }; let required_ordering = new_plan .output_ordering() - .map(PhysicalSortRequirement::from_sort_exprs); + .map(PhysicalSortRequirement::from_sort_exprs) + .unwrap_or_default(); // Since new_plan is a SortExec, we can safely get the 0th index. let child = new_plan.children().swap_remove(0); if let Some(adjusted) = - pushdown_requirement_to_children(&child, required_ordering.as_deref())? + pushdown_requirement_to_children(&child, &required_ordering)? { // Can push down requirements Ok(Transformed::Yes(SortPushDown { @@ -163,12 +158,10 @@ pub(crate) fn pushdown_sorts( } } else { // Executors other than SortExec - if ordering_satisfy_requirement( - plan.output_ordering(), - parent_required, - || plan.equivalence_properties(), - || plan.ordering_equivalence_properties(), - ) { + if plan + .equivalence_properties() + .ordering_satisfy_requirement(parent_required) + { // Satisfies parent requirements, immediately return. return Ok(Transformed::Yes(SortPushDown { required_ordering: None, @@ -184,9 +177,8 @@ pub(crate) fn pushdown_sorts( })) } else { // Can not push down requirements, add new SortExec: - let parent_required_expr = parent_required.ok_or_else(err)?; let mut new_plan = requirements.plan; - add_sort_above(&mut new_plan, parent_required_expr, None); + add_sort_above(&mut new_plan, parent_required, None); Ok(Transformed::Yes(SortPushDown::init(new_plan))) } } @@ -194,18 +186,21 @@ pub(crate) fn pushdown_sorts( fn pushdown_requirement_to_children( plan: &Arc, - parent_required: Option<&[PhysicalSortRequirement]>, + parent_required: LexRequirementRef, ) -> Result>>>> { - const ERR_MSG: &str = "Expects parent requirement to contain something"; - let err = || plan_datafusion_err!("{}", ERR_MSG); let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); - let request_child = required_input_ordering[0].as_deref(); + let request_child = required_input_ordering[0].as_deref().unwrap_or(&[]); let child_plan = plan.children().swap_remove(0); match determine_children_requirement(parent_required, request_child, child_plan) { RequirementsCompatibility::Satisfy => { - Ok(Some(vec![request_child.map(|r| r.to_vec())])) + let req = if request_child.is_empty() { + None + } else { + Some(request_child.to_vec()) + }; + Ok(Some(vec![req])) } RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), RequirementsCompatibility::NonCompatible => Ok(None), @@ -213,16 +208,17 @@ fn pushdown_requirement_to_children( } else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec - Ok(Some(vec![ - parent_required.map(|elem| elem.to_vec()); - plan.children().len() - ])) + let req = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; + Ok(Some(vec![req; plan.children().len()])) } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec let left_columns_len = smj.left().schema().fields().len(); - let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( - parent_required.ok_or_else(err)?.iter().cloned(), - ); + let parent_required_expr = + PhysicalSortRequirement::to_sort_exprs(parent_required.iter().cloned()); let expr_source_side = expr_source_sides(&parent_required_expr, smj.join_type(), left_columns_len); match expr_source_side { @@ -236,10 +232,9 @@ fn pushdown_requirement_to_children( let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); let new_right_required = - shift_right_required(parent_required.ok_or_else(err)?, right_offset)?; - let new_right_required_expr = PhysicalSortRequirement::to_sort_exprs( - new_right_required.iter().cloned(), - ); + shift_right_required(parent_required, right_offset)?; + let new_right_required_expr = + PhysicalSortRequirement::to_sort_exprs(new_right_required); try_pushdown_requirements_to_join( smj, parent_required, @@ -260,34 +255,43 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() || is_limit(plan) || plan.as_any().is::() - // Do not push-down through SortPreservingMergeExec when - // ordering requirement invalidates requirement of sort preserving merge exec. - || (is_sort_preserving_merge(plan) && !ordering_satisfy( - parent_required - .map(|req| PhysicalSortRequirement::to_sort_exprs(req.to_vec())) - .as_deref(), - plan.output_ordering(), - || plan.equivalence_properties(), - || plan.ordering_equivalence_properties(), - ) - ) { // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. // Pushing down is not beneficial Ok(None) + } 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(); + // 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 + // ordering requirement invalidates requirement of sort preserving merge exec. + if !spm_eqs.ordering_satisfy(plan.output_ordering().unwrap_or(&[])) { + Ok(None) + } else { + // Can push-down through SortPreservingMergeExec, because parent requirement is finer + // than SortPreservingMergeExec output ordering. + let req = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; + Ok(Some(vec![req])) + } } else { Ok(Some( maintains_input_order - .iter() + .into_iter() .map(|flag| { - if *flag { - parent_required.map(|elem| elem.to_vec()) + if flag && !parent_required.is_empty() { + Some(parent_required.to_vec()) } else { None } }) - .collect::>(), + .collect(), )) } // TODO: Add support for Projection push down @@ -298,26 +302,26 @@ fn pushdown_requirement_to_children( /// If the the parent requirements are more specific, push down the parent requirements /// If they are not compatible, need to add Sort. fn determine_children_requirement( - parent_required: Option<&[PhysicalSortRequirement]>, - request_child: Option<&[PhysicalSortRequirement]>, + parent_required: LexRequirementRef, + request_child: LexRequirementRef, child_plan: Arc, ) -> RequirementsCompatibility { - if requirements_compatible( - request_child, - parent_required, - || child_plan.ordering_equivalence_properties(), - || child_plan.equivalence_properties(), - ) { + if child_plan + .equivalence_properties() + .requirements_compatible(request_child, parent_required) + { // request child requirements are more specific, no need to push down the parent requirements RequirementsCompatibility::Satisfy - } else if requirements_compatible( - parent_required, - request_child, - || child_plan.ordering_equivalence_properties(), - || child_plan.equivalence_properties(), - ) { + } else if child_plan + .equivalence_properties() + .requirements_compatible(parent_required, request_child) + { // parent requirements are more specific, adjust the request child requirements and push down the new requirements - let adjusted = parent_required.map(|r| r.to_vec()); + let adjusted = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; RequirementsCompatibility::Compatible(adjusted) } else { RequirementsCompatibility::NonCompatible @@ -325,7 +329,7 @@ fn determine_children_requirement( } fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, - parent_required: Option<&[PhysicalSortRequirement]>, + parent_required: LexRequirementRef, sort_expr: Vec, push_side: JoinSide, ) -> Result>>>> { @@ -335,22 +339,22 @@ fn try_pushdown_requirements_to_join( JoinSide::Left => (sort_expr.as_slice(), right_ordering), JoinSide::Right => (left_ordering, sort_expr.as_slice()), }; + let join_type = smj.join_type(); + let probe_side = SortMergeJoinExec::probe_side(&join_type); let new_output_ordering = calculate_join_output_ordering( new_left_ordering, new_right_ordering, - smj.join_type(), + join_type, smj.on(), smj.left().schema().fields.len(), &smj.maintains_input_order(), - Some(SortMergeJoinExec::probe_side(&smj.join_type())), + Some(probe_side), ); - Ok(ordering_satisfy_requirement( - new_output_ordering.as_deref(), - parent_required, - || smj.equivalence_properties(), - || smj.ordering_equivalence_properties(), - ) - .then(|| { + let mut smj_eqs = smj.equivalence_properties(); + // 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); + Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); match push_side { @@ -414,7 +418,7 @@ fn expr_source_sides( } fn shift_right_required( - parent_required: &[PhysicalSortRequirement], + parent_required: LexRequirementRef, left_columns_len: usize, ) -> Result> { let new_right_required: Vec = parent_required diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 403af4b16ec7..530df374ca7c 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -30,8 +30,7 @@ use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{displayable, ExecutionPlan}; -use datafusion_physical_expr::utils::ordering_satisfy_requirement; -use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr::{LexRequirementRef, PhysicalSortRequirement}; /// This object implements a tree that we use while keeping track of paths /// leading to [`SortExec`]s. @@ -100,16 +99,14 @@ pub(crate) fn get_children_exectrees( /// given ordering requirements while preserving the original partitioning. pub fn add_sort_above( node: &mut Arc, - sort_requirement: &[PhysicalSortRequirement], + sort_requirement: LexRequirementRef, fetch: Option, ) { // If the ordering requirement is already satisfied, do not add a sort. - if !ordering_satisfy_requirement( - node.output_ordering(), - Some(sort_requirement), - || node.equivalence_properties(), - || node.ordering_equivalence_properties(), - ) { + if !node + .equivalence_properties() + .ordering_satisfy_requirement(sort_requirement) + { let sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirement.to_vec()); let new_sort = SortExec::new(sort_expr, node.clone()).with_fetch(fetch); diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 798f7ae8baf6..821f236af87b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -35,39 +35,33 @@ use datafusion_physical_expr::expressions::{col, Sum}; use datafusion_physical_expr::{AggregateExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; -#[cfg(test)] -#[allow(clippy::items_after_test_module)] -mod tests { - use super::*; - - #[tokio::test(flavor = "multi_thread", worker_threads = 8)] - async fn aggregate_test() { - let test_cases = vec![ - vec!["a"], - vec!["b", "a"], - vec!["c", "a"], - vec!["c", "b", "a"], - vec!["d", "a"], - vec!["d", "b", "a"], - vec!["d", "c", "a"], - vec!["d", "c", "b", "a"], - ]; - let n = 300; - let distincts = vec![10, 20]; - for distinct in distincts { - let mut handles = Vec::new(); - for i in 0..n { - let test_idx = i % test_cases.len(); - let group_by_columns = test_cases[test_idx].clone(); - let job = tokio::spawn(run_aggregate_test( - make_staggered_batches::(1000, distinct, i as u64), - group_by_columns, - )); - handles.push(job); - } - for job in handles { - job.await.unwrap(); - } +#[tokio::test(flavor = "multi_thread", worker_threads = 8)] +async fn aggregate_test() { + let test_cases = vec![ + vec!["a"], + vec!["b", "a"], + vec!["c", "a"], + vec!["c", "b", "a"], + vec!["d", "a"], + vec!["d", "b", "a"], + vec!["d", "c", "a"], + vec!["d", "c", "b", "a"], + ]; + let n = 300; + let distincts = vec![10, 20]; + for distinct in distincts { + let mut handles = Vec::new(); + for i in 0..n { + let test_idx = i % test_cases.len(); + let group_by_columns = test_cases[test_idx].clone(); + let job = tokio::spawn(run_aggregate_test( + make_staggered_batches::(1000, distinct, i as u64), + group_by_columns, + )); + handles.push(job); + } + for job in handles { + job.await.unwrap(); } } } diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 66f7374a51fb..af96063ffb5f 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -22,7 +22,6 @@ use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; - use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ @@ -44,108 +43,102 @@ use hashbrown::HashMap; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; -#[cfg(test)] -#[allow(clippy::items_after_test_module)] -mod tests { - use super::*; - - use datafusion_physical_plan::windows::PartitionSearchMode::{ - Linear, PartiallySorted, Sorted, - }; +use datafusion_physical_plan::windows::PartitionSearchMode::{ + Linear, PartiallySorted, Sorted, +}; - #[tokio::test(flavor = "multi_thread", worker_threads = 16)] - async fn window_bounded_window_random_comparison() -> Result<()> { - // make_staggered_batches gives result sorted according to a, b, c - // In the test cases first entry represents partition by columns - // Second entry represents order by columns. - // Third entry represents search mode. - // In sorted mode physical plans are in the form for WindowAggExec - //``` - // WindowAggExec - // MemoryExec] - // ``` - // and in the form for BoundedWindowAggExec - // ``` - // BoundedWindowAggExec - // MemoryExec - // ``` - // In Linear and PartiallySorted mode physical plans are in the form for WindowAggExec - //``` - // WindowAggExec - // SortExec(required by window function) - // MemoryExec] - // ``` - // and in the form for BoundedWindowAggExec - // ``` - // BoundedWindowAggExec - // MemoryExec - // ``` - let test_cases = vec![ - (vec!["a"], vec!["a"], Sorted), - (vec!["a"], vec!["b"], Sorted), - (vec!["a"], vec!["a", "b"], Sorted), - (vec!["a"], vec!["b", "c"], Sorted), - (vec!["a"], vec!["a", "b", "c"], Sorted), - (vec!["b"], vec!["a"], Linear), - (vec!["b"], vec!["a", "b"], Linear), - (vec!["b"], vec!["a", "c"], Linear), - (vec!["b"], vec!["a", "b", "c"], Linear), - (vec!["c"], vec!["a"], Linear), - (vec!["c"], vec!["a", "b"], Linear), - (vec!["c"], vec!["a", "c"], Linear), - (vec!["c"], vec!["a", "b", "c"], Linear), - (vec!["b", "a"], vec!["a"], Sorted), - (vec!["b", "a"], vec!["b"], Sorted), - (vec!["b", "a"], vec!["c"], Sorted), - (vec!["b", "a"], vec!["a", "b"], Sorted), - (vec!["b", "a"], vec!["b", "c"], Sorted), - (vec!["b", "a"], vec!["a", "c"], Sorted), - (vec!["b", "a"], vec!["a", "b", "c"], Sorted), - (vec!["c", "b"], vec!["a"], Linear), - (vec!["c", "b"], vec!["a", "b"], Linear), - (vec!["c", "b"], vec!["a", "c"], Linear), - (vec!["c", "b"], vec!["a", "b", "c"], Linear), - (vec!["c", "a"], vec!["a"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["b"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["c"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["a", "b"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["b", "c"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["a", "c"], PartiallySorted(vec![1])), - ( - vec!["c", "a"], - vec!["a", "b", "c"], - PartiallySorted(vec![1]), - ), - (vec!["c", "b", "a"], vec!["a"], Sorted), - (vec!["c", "b", "a"], vec!["b"], Sorted), - (vec!["c", "b", "a"], vec!["c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "b"], Sorted), - (vec!["c", "b", "a"], vec!["b", "c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "b", "c"], Sorted), - ]; - let n = 300; - let n_distincts = vec![10, 20]; - for n_distinct in n_distincts { - let mut handles = Vec::new(); - for i in 0..n { - let idx = i % test_cases.len(); - let (pb_cols, ob_cols, search_mode) = test_cases[idx].clone(); - let job = tokio::spawn(run_window_test( - make_staggered_batches::(1000, n_distinct, i as u64), - i as u64, - pb_cols, - ob_cols, - search_mode, - )); - handles.push(job); - } - for job in handles { - job.await.unwrap()?; - } +#[tokio::test(flavor = "multi_thread", worker_threads = 16)] +async fn window_bounded_window_random_comparison() -> Result<()> { + // make_staggered_batches gives result sorted according to a, b, c + // In the test cases first entry represents partition by columns + // Second entry represents order by columns. + // Third entry represents search mode. + // In sorted mode physical plans are in the form for WindowAggExec + //``` + // WindowAggExec + // MemoryExec] + // ``` + // and in the form for BoundedWindowAggExec + // ``` + // BoundedWindowAggExec + // MemoryExec + // ``` + // In Linear and PartiallySorted mode physical plans are in the form for WindowAggExec + //``` + // WindowAggExec + // SortExec(required by window function) + // MemoryExec] + // ``` + // and in the form for BoundedWindowAggExec + // ``` + // BoundedWindowAggExec + // MemoryExec + // ``` + let test_cases = vec![ + (vec!["a"], vec!["a"], Sorted), + (vec!["a"], vec!["b"], Sorted), + (vec!["a"], vec!["a", "b"], Sorted), + (vec!["a"], vec!["b", "c"], Sorted), + (vec!["a"], vec!["a", "b", "c"], Sorted), + (vec!["b"], vec!["a"], Linear), + (vec!["b"], vec!["a", "b"], Linear), + (vec!["b"], vec!["a", "c"], Linear), + (vec!["b"], vec!["a", "b", "c"], Linear), + (vec!["c"], vec!["a"], Linear), + (vec!["c"], vec!["a", "b"], Linear), + (vec!["c"], vec!["a", "c"], Linear), + (vec!["c"], vec!["a", "b", "c"], Linear), + (vec!["b", "a"], vec!["a"], Sorted), + (vec!["b", "a"], vec!["b"], Sorted), + (vec!["b", "a"], vec!["c"], Sorted), + (vec!["b", "a"], vec!["a", "b"], Sorted), + (vec!["b", "a"], vec!["b", "c"], Sorted), + (vec!["b", "a"], vec!["a", "c"], Sorted), + (vec!["b", "a"], vec!["a", "b", "c"], Sorted), + (vec!["c", "b"], vec!["a"], Linear), + (vec!["c", "b"], vec!["a", "b"], Linear), + (vec!["c", "b"], vec!["a", "c"], Linear), + (vec!["c", "b"], vec!["a", "b", "c"], Linear), + (vec!["c", "a"], vec!["a"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["b"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["c"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["a", "b"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["b", "c"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["a", "c"], PartiallySorted(vec![1])), + ( + vec!["c", "a"], + vec!["a", "b", "c"], + PartiallySorted(vec![1]), + ), + (vec!["c", "b", "a"], vec!["a"], Sorted), + (vec!["c", "b", "a"], vec!["b"], Sorted), + (vec!["c", "b", "a"], vec!["c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "b"], Sorted), + (vec!["c", "b", "a"], vec!["b", "c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "b", "c"], Sorted), + ]; + let n = 300; + let n_distincts = vec![10, 20]; + for n_distinct in n_distincts { + let mut handles = Vec::new(); + for i in 0..n { + let idx = i % test_cases.len(); + let (pb_cols, ob_cols, search_mode) = test_cases[idx].clone(); + let job = tokio::spawn(run_window_test( + make_staggered_batches::(1000, n_distinct, i as u64), + i as u64, + pb_cols, + ob_cols, + search_mode, + )); + handles.push(job); + } + for job in handles { + job.await.unwrap()?; } - Ok(()) } + Ok(()) } fn get_random_function( diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 4fce6854138d..04b0f2eedcdb 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -15,148 +15,200 @@ // specific language governing permissions and limitations // under the License. -use crate::expressions::{CastExpr, Column}; -use crate::utils::{collect_columns, merge_vectors}; +use std::collections::HashSet; +use std::hash::Hash; +use std::sync::Arc; + +use crate::expressions::Column; +use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; +use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::{ - LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, + physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, + LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; -use arrow_schema::Fields; - +use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{JoinSide, JoinType}; -use itertools::izip; -use std::collections::{HashMap, HashSet}; -use std::hash::Hash; -use std::ops::Range; -use std::sync::Arc; +use datafusion_common::{JoinSide, JoinType, Result}; -/// Represents a collection of [`EquivalentClass`] (equivalences -/// between columns in relations) -/// -/// This is used to represent: -/// -/// 1. Equality conditions (like `A=B`), when `T` = [`Column`] +use indexmap::map::Entry; +use indexmap::IndexMap; + +/// An `EquivalenceClass` is a set of [`Arc`]s that are known +/// to have the same value for all tuples in a relation. These are generated by +/// equality predicates, typically equi-join conditions and equality conditions +/// in filters. +pub type EquivalenceClass = Vec>; + +/// Stores the mapping between source expressions and target expressions for a +/// projection. Indices in the vector corresponds to the indices after projection. +pub type ProjectionMapping = Vec<(Arc, Arc)>; + +/// An `EquivalenceGroup` is a collection of `EquivalenceClass`es where each +/// class represents a distinct equivalence class in a relation. #[derive(Debug, Clone)] -pub struct EquivalenceProperties { - classes: Vec>, - schema: SchemaRef, +pub struct EquivalenceGroup { + classes: Vec, } -impl EquivalenceProperties { - pub fn new(schema: SchemaRef) -> Self { - EquivalenceProperties { - classes: vec![], - schema, - } +impl EquivalenceGroup { + /// Creates an empty equivalence group. + fn empty() -> Self { + Self { classes: vec![] } } - /// return the set of equivalences - pub fn classes(&self) -> &[EquivalentClass] { - &self.classes + /// Creates an equivalence group from the given equivalence classes. + fn new(classes: Vec) -> Self { + let mut result = EquivalenceGroup { classes }; + result.remove_redundant_entries(); + result } - pub fn schema(&self) -> SchemaRef { - self.schema.clone() + /// Returns how many equivalence classes there are in this group. + fn len(&self) -> usize { + self.classes.len() } - /// Add the [`EquivalentClass`] from `iter` to this list - pub fn extend>>(&mut self, iter: I) { - for ec in iter { - self.classes.push(ec) - } + /// Checks whether this equivalence group is empty. + pub fn is_empty(&self) -> bool { + self.len() == 0 } - /// Adds new equal conditions into the EquivalenceProperties. New equal - /// conditions usually come from equality predicates in a join/filter. - pub fn add_equal_conditions(&mut self, new_conditions: (&Column, &Column)) { - let mut idx1: Option = None; - let mut idx2: Option = None; - for (idx, class) in self.classes.iter_mut().enumerate() { - let contains_first = class.contains(new_conditions.0); - let contains_second = class.contains(new_conditions.1); - match (contains_first, contains_second) { - (true, false) => { - class.insert(new_conditions.1.clone()); - idx1 = Some(idx); - } - (false, true) => { - class.insert(new_conditions.0.clone()); - idx2 = Some(idx); - } - (true, true) => { - idx1 = Some(idx); - idx2 = Some(idx); - break; - } - (false, false) => {} + /// Returns an iterator over the equivalence classes in this group. + fn iter(&self) -> impl Iterator { + self.classes.iter() + } + + /// Adds the equality `left` = `right` to this equivalence group. + /// New equality conditions often arise after steps like `Filter(a = b)`, + /// `Alias(a, a as b)` etc. + fn add_equal_conditions( + &mut self, + left: &Arc, + right: &Arc, + ) { + let mut first_class = None; + let mut second_class = None; + for (idx, cls) in self.classes.iter().enumerate() { + if physical_exprs_contains(cls, left) { + first_class = Some(idx); + } + if physical_exprs_contains(cls, right) { + second_class = Some(idx); } } - - match (idx1, idx2) { - (Some(idx_1), Some(idx_2)) if idx_1 != idx_2 => { - // need to merge the two existing EquivalentClasses - let second_eq_class = self.classes.get(idx_2).unwrap().clone(); - let first_eq_class = self.classes.get_mut(idx_1).unwrap(); - for prop in second_eq_class.iter() { - if !first_eq_class.contains(prop) { - first_eq_class.insert(prop.clone()); + match (first_class, second_class) { + (Some(mut first_idx), Some(mut second_idx)) => { + // If the given left and right sides belong to different classes, + // we should unify/bridge these classes. + if first_idx != second_idx { + // By convention make sure second_idx is larger than first_idx. + if first_idx > second_idx { + (first_idx, second_idx) = (second_idx, first_idx); } + // Remove second_idx from self.classes then merge its values with class at first_idx. + // Convention above makes sure that first_idx is still valid after second_idx removal. + let other_class = self.classes.swap_remove(second_idx); + self.classes[first_idx].extend(other_class); } - self.classes.remove(idx_2); + } + (Some(group_idx), None) => { + // Right side is new, extend left side's class: + self.classes[group_idx].push(right.clone()); + } + (None, Some(group_idx)) => { + // Left side is new, extend right side's class: + self.classes[group_idx].push(left.clone()); } (None, None) => { - // adding new pairs - self.classes.push(EquivalentClass::::new( - new_conditions.0.clone(), - vec![new_conditions.1.clone()], - )); + // None of the expressions is among existing classes. + // Create a new equivalence class and extend the group. + self.classes.push(vec![left.clone(), right.clone()]); } - _ => {} } } - /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. - /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. + /// Removes redundant entries from this group. + fn remove_redundant_entries(&mut self) { + // Remove duplicate entries from each equivalence class: + self.classes.retain_mut(|cls| { + // Keep groups that have at least two entries as singleton class is + // meaningless (i.e. it contains no non-trivial information): + deduplicate_physical_exprs(cls); + cls.len() > 1 + }); + // Unify/bridge groups that have common expressions: + self.bridge_classes() + } + + /// This utility function unifies/bridges classes that have common expressions. + /// For example, assume that we have [`EquivalenceClass`]es `[a, b]` and `[b, c]`. + /// Since both classes contain `b`, columns `a`, `b` and `c` are actually all + /// equal and belong to one class. This utility converts merges such classes. + fn bridge_classes(&mut self) { + let mut idx = 0; + while idx < self.classes.len() { + let mut next_idx = idx + 1; + let start_size = self.classes[idx].len(); + while next_idx < self.classes.len() { + if have_common_entries(&self.classes[idx], &self.classes[next_idx]) { + let extension = self.classes.swap_remove(next_idx); + self.classes[idx].extend(extension); + } else { + next_idx += 1; + } + } + if self.classes[idx].len() > start_size { + deduplicate_physical_exprs(&mut self.classes[idx]); + if self.classes[idx].len() > start_size { + continue; + } + } + idx += 1; + } + } + + /// Extends this equivalence group with the `other` equivalence group. + fn extend(&mut self, other: Self) { + self.classes.extend(other.classes); + self.remove_redundant_entries(); + } + + /// Normalizes the given physical expression according to this group. + /// The expression is replaced with the first expression in the equivalence + /// class it matches with (if any). pub fn normalize_expr(&self, expr: Arc) -> Arc { expr.clone() .transform(&|expr| { - let normalized_form = - expr.as_any().downcast_ref::().and_then(|column| { - for class in &self.classes { - if class.contains(column) { - return Some(Arc::new(class.head().clone()) as _); - } - } - None - }); - Ok(if let Some(normalized_form) = normalized_form { - Transformed::Yes(normalized_form) - } else { - Transformed::No(expr) - }) + for cls in self.iter() { + if physical_exprs_contains(cls, &expr) { + return Ok(Transformed::Yes(cls[0].clone())); + } + } + Ok(Transformed::No(expr)) }) .unwrap_or(expr) } - /// This function applies the \[`normalize_expr`] - /// function for all expression in `exprs` and returns a vector of - /// normalized physical expressions. - pub fn normalize_exprs( + /// Normalizes the given sort expression according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the sort expression as is. + pub fn normalize_sort_expr( &self, - exprs: &[Arc], - ) -> Vec> { - exprs - .iter() - .map(|expr| self.normalize_expr(expr.clone())) - .collect::>() + mut sort_expr: PhysicalSortExpr, + ) -> PhysicalSortExpr { + sort_expr.expr = self.normalize_expr(sort_expr.expr); + sort_expr } - /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. - /// If the given sort requirement doesn't belong to equivalence set inside - /// `self`, it returns `sort_requirement` as is. + /// Normalizes the given sort requirement according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the given sort requirement as is. pub fn normalize_sort_requirement( &self, mut sort_requirement: PhysicalSortRequirement, @@ -165,1069 +217,1145 @@ impl EquivalenceProperties { sort_requirement } - /// This function applies the \[`normalize_sort_requirement`] - /// function for all sort requirements in `sort_reqs` and returns a vector of - /// normalized sort expressions. - pub fn normalize_sort_requirements( + /// This function applies the `normalize_expr` function for all expressions + /// in `exprs` and returns the corresponding normalized physical expressions. + pub fn normalize_exprs( &self, - sort_reqs: &[PhysicalSortRequirement], - ) -> Vec { - let normalized_sort_reqs = sort_reqs - .iter() - .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) - .collect::>(); - collapse_vec(normalized_sort_reqs) + exprs: impl IntoIterator>, + ) -> Vec> { + exprs + .into_iter() + .map(|expr| self.normalize_expr(expr)) + .collect() } - /// Similar to the \[`normalize_sort_requirements`] this function normalizes - /// sort expressions in `sort_exprs` and returns a vector of - /// normalized sort expressions. - pub fn normalize_sort_exprs( - &self, - sort_exprs: &[PhysicalSortExpr], - ) -> Vec { - let sort_requirements = - PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_sort_requirement = - self.normalize_sort_requirements(&sort_requirements); - PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) + /// This function applies the `normalize_sort_expr` function for all sort + /// expressions in `sort_exprs` and returns the corresponding normalized + /// sort expressions. + pub fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + // Convert sort expressions to sort requirements: + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + // Normalize the requirements: + let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); + // Convert sort requirements back to sort expressions: + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) } -} - -/// `OrderingEquivalenceProperties` keeps track of columns that describe the -/// global ordering of the schema. These columns are not necessarily same; e.g. -/// ```text -/// ┌-------┐ -/// | a | b | -/// |---|---| -/// | 1 | 9 | -/// | 2 | 8 | -/// | 3 | 7 | -/// | 5 | 5 | -/// └---┴---┘ -/// ``` -/// where both `a ASC` and `b DESC` can describe the table ordering. With -/// `OrderingEquivalenceProperties`, we can keep track of these equivalences -/// and treat `a ASC` and `b DESC` as the same ordering requirement. -#[derive(Debug, Clone)] -pub struct OrderingEquivalenceProperties { - oeq_class: Option, - /// Keeps track of expressions that have constant value. - constants: Vec>, - schema: SchemaRef, -} -impl OrderingEquivalenceProperties { - /// Create an empty `OrderingEquivalenceProperties` - pub fn new(schema: SchemaRef) -> Self { - Self { - oeq_class: None, - constants: vec![], - schema, - } + /// This function applies the `normalize_sort_requirement` function for all + /// requirements in `sort_reqs` and returns the corresponding normalized + /// sort requirements. + pub fn normalize_sort_requirements( + &self, + sort_reqs: LexRequirementRef, + ) -> LexRequirement { + collapse_lex_req( + sort_reqs + .iter() + .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) + .collect(), + ) } - /// Extends `OrderingEquivalenceProperties` by adding ordering inside the `other` - /// to the `self.oeq_class`. - pub fn extend(&mut self, other: Option) { - if let Some(other) = other { - if let Some(class) = &mut self.oeq_class { - class.others.insert(other.head); - class.others.extend(other.others); - } else { - self.oeq_class = Some(other); + /// Projects `expr` according to the given projection mapping. + /// If the resulting expression is invalid after projection, returns `None`. + fn project_expr( + &self, + mapping: &ProjectionMapping, + expr: &Arc, + ) -> Option> { + let children = expr.children(); + if children.is_empty() { + for (source, target) in mapping.iter() { + // If we match the source, or an equivalent expression to source, + // then we can project. For example, if we have the mapping + // (a as a1, a + c) and the equivalence class (a, b), expression + // b also projects to a1. + if source.eq(expr) + || self + .get_equivalence_class(source) + .map_or(false, |group| physical_exprs_contains(group, expr)) + { + return Some(target.clone()); + } } } + // Project a non-leaf expression by projecting its children. + else if let Some(children) = children + .into_iter() + .map(|child| self.project_expr(mapping, &child)) + .collect::>>() + { + return Some(expr.clone().with_new_children(children).unwrap()); + } + // Arriving here implies the expression was invalid after projection. + None } - pub fn oeq_class(&self) -> Option<&OrderingEquivalentClass> { - self.oeq_class.as_ref() - } - - /// Adds new equal conditions into the EquivalenceProperties. New equal - /// conditions usually come from equality predicates in a join/filter. - pub fn add_equal_conditions(&mut self, new_conditions: (&LexOrdering, &LexOrdering)) { - if let Some(class) = &mut self.oeq_class { - class.insert(new_conditions.0.clone()); - class.insert(new_conditions.1.clone()); - } else { - let head = new_conditions.0.clone(); - let others = vec![new_conditions.1.clone()]; - self.oeq_class = Some(OrderingEquivalentClass::new(head, others)) - } + /// Projects `ordering` according to the given projection mapping. + /// If the resulting ordering is invalid after projection, returns `None`. + fn project_ordering( + &self, + mapping: &ProjectionMapping, + ordering: LexOrderingRef, + ) -> Option { + // If any sort expression is invalid after projection, rest of the + // ordering shouldn't be projected either. For example, if input ordering + // is [a ASC, b ASC, c ASC], and column b is not valid after projection, + // the result should be [a ASC], not [a ASC, c ASC], even if column c is + // valid after projection. + let result = ordering + .iter() + .map_while(|sort_expr| { + self.project_expr(mapping, &sort_expr.expr) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }) + }) + .collect::>(); + (!result.is_empty()).then_some(result) } - /// Add physical expression that have constant value to the `self.constants` - pub fn with_constants(mut self, constants: Vec>) -> Self { - constants.into_iter().for_each(|constant| { - if !physical_exprs_contains(&self.constants, &constant) { - self.constants.push(constant); - } + /// Projects this equivalence group according to the given projection mapping. + pub fn project(&self, mapping: &ProjectionMapping) -> Self { + let projected_classes = self.iter().filter_map(|cls| { + let new_class = cls + .iter() + .filter_map(|expr| self.project_expr(mapping, expr)) + .collect::>(); + (new_class.len() > 1).then_some(new_class) }); - self + // TODO: Convert the algorithm below to a version that uses `HashMap`. + // once `Arc` can be stored in `HashMap`. + // See issue: https://github.com/apache/arrow-datafusion/issues/8027 + let mut new_classes = vec![]; + for (source, target) in mapping { + if new_classes.is_empty() { + new_classes.push((source, vec![target.clone()])); + } + if let Some((_, values)) = + new_classes.iter_mut().find(|(key, _)| key.eq(source)) + { + if !physical_exprs_contains(values, target) { + values.push(target.clone()); + } + } + } + // Only add equivalence classes with at least two members as singleton + // equivalence classes are meaningless. + let new_classes = new_classes + .into_iter() + .filter_map(|(_, values)| (values.len() > 1).then_some(values)); + let classes = projected_classes.chain(new_classes).collect(); + Self::new(classes) } - pub fn schema(&self) -> SchemaRef { - self.schema.clone() + /// Returns the equivalence class that contains `expr`. + /// If none of the equivalence classes contains `expr`, returns `None`. + fn get_equivalence_class( + &self, + expr: &Arc, + ) -> Option<&[Arc]> { + self.iter() + .map(|cls| cls.as_slice()) + .find(|cls| physical_exprs_contains(cls, expr)) } - /// This function normalizes `sort_reqs` by - /// - removing expressions that have constant value from requirement - /// - replacing sections that are in the `self.oeq_class.others` with `self.oeq_class.head` - /// - removing sections that satisfies global ordering that are in the post fix of requirement - pub fn normalize_sort_requirements( + /// Combine equivalence groups of the given join children. + pub fn join( &self, - sort_reqs: &[PhysicalSortRequirement], - ) -> Vec { - let normalized_sort_reqs = - prune_sort_reqs_with_constants(sort_reqs, &self.constants); - let mut normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); - if let Some(oeq_class) = &self.oeq_class { - for item in oeq_class.others() { - let item = PhysicalSortRequirement::from_sort_exprs(item); - let item = prune_sort_reqs_with_constants(&item, &self.constants); - let ranges = get_compatible_ranges(&normalized_sort_reqs, &item); - let mut offset: i64 = 0; - for Range { start, end } in ranges { - let head = PhysicalSortRequirement::from_sort_exprs(oeq_class.head()); - let mut head = prune_sort_reqs_with_constants(&head, &self.constants); - let updated_start = (start as i64 + offset) as usize; - let updated_end = (end as i64 + offset) as usize; - let range = end - start; - offset += head.len() as i64 - range as i64; - let all_none = normalized_sort_reqs[updated_start..updated_end] - .iter() - .all(|req| req.options.is_none()); - if all_none { - for req in head.iter_mut() { - req.options = None; - } + right_equivalences: &Self, + join_type: &JoinType, + left_size: usize, + on: &[(Column, Column)], + ) -> Self { + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + let mut result = Self::new( + self.iter() + .cloned() + .chain(right_equivalences.iter().map(|item| { + item.iter() + .cloned() + .map(|expr| add_offset_to_expr(expr, left_size)) + .collect() + })) + .collect(), + ); + // In we have an inner join, expressions in the "on" condition + // are equal in the resulting table. + if join_type == &JoinType::Inner { + for (lhs, rhs) in on.iter() { + let index = rhs.index() + left_size; + let new_lhs = Arc::new(lhs.clone()) as _; + let new_rhs = Arc::new(Column::new(rhs.name(), index)) as _; + result.add_equal_conditions(&new_lhs, &new_rhs); } - normalized_sort_reqs.splice(updated_start..updated_end, head); } + result } - normalized_sort_reqs = simplify_lex_req(normalized_sort_reqs, oeq_class); + JoinType::LeftSemi | JoinType::LeftAnti => self.clone(), + JoinType::RightSemi | JoinType::RightAnti => right_equivalences.clone(), } - collapse_lex_req(normalized_sort_reqs) } +} - /// Checks whether `leading_ordering` is contained in any of the ordering - /// equivalence classes. - pub fn satisfies_leading_ordering( - &self, - leading_ordering: &PhysicalSortExpr, - ) -> bool { - if let Some(oeq_class) = &self.oeq_class { - for ordering in oeq_class - .others - .iter() - .chain(std::iter::once(&oeq_class.head)) - { - if ordering[0].eq(leading_ordering) { - return true; - } - } +/// This function constructs a duplicate-free `LexOrderingReq` by filtering out +/// duplicate entries that have same physical expression inside. For example, +/// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. +pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { + let mut output = Vec::::new(); + for item in input { + if !output.iter().any(|req| req.expr.eq(&item.expr)) { + output.push(item); } - false } + output } -/// EquivalentClass is a set of [`Column`]s or [`PhysicalSortExpr`]s that are known -/// to have the same value in all tuples in a relation. `EquivalentClass` -/// is generated by equality predicates, typically equijoin conditions and equality -/// conditions in filters. `EquivalentClass` is generated by the -/// `ROW_NUMBER` window function. -#[derive(Debug, Clone)] -pub struct EquivalentClass { - /// First element in the EquivalentClass - head: T, - /// Other equal columns - others: HashSet, +/// An `OrderingEquivalenceClass` object keeps track of different alternative +/// orderings than can describe a schema. For example, consider the following table: +/// +/// ```text +/// |a|b|c|d| +/// |1|4|3|1| +/// |2|3|3|2| +/// |3|1|2|2| +/// |3|2|1|3| +/// ``` +/// +/// Here, both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the table +/// ordering. In this case, we say that these orderings are equivalent. +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct OrderingEquivalenceClass { + orderings: Vec, } -impl EquivalentClass { - pub fn new(head: T, others: Vec) -> EquivalentClass { - EquivalentClass { - head, - others: HashSet::from_iter(others), - } +impl OrderingEquivalenceClass { + /// Creates new empty ordering equivalence class. + fn empty() -> Self { + Self { orderings: vec![] } } - pub fn head(&self) -> &T { - &self.head + /// Clears (empties) this ordering equivalence class. + pub fn clear(&mut self) { + self.orderings.clear(); } - pub fn others(&self) -> &HashSet { - &self.others + /// Creates new ordering equivalence class from the given orderings. + pub fn new(orderings: Vec) -> Self { + let mut result = Self { orderings }; + result.remove_redundant_entries(); + result } - pub fn contains(&self, col: &T) -> bool { - self.head == *col || self.others.contains(col) + /// Checks whether `ordering` is a member of this equivalence class. + pub fn contains(&self, ordering: &LexOrdering) -> bool { + self.orderings.contains(ordering) } - pub fn insert(&mut self, col: T) -> bool { - self.head != col && self.others.insert(col) + /// Adds `ordering` to this equivalence class. + #[allow(dead_code)] + fn push(&mut self, ordering: LexOrdering) { + self.orderings.push(ordering); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); } - pub fn remove(&mut self, col: &T) -> bool { - let removed = self.others.remove(col); - // If we are removing the head, adjust others so that its first entry becomes the new head. - if !removed && *col == self.head { - if let Some(col) = self.others.iter().next().cloned() { - let removed = self.others.remove(&col); - self.head = col; - removed - } else { - // We don't allow empty equivalence classes, reject removal if one tries removing - // the only element in an equivalence class. - false - } - } else { - removed - } + /// Checks whether this ordering equivalence class is empty. + pub fn is_empty(&self) -> bool { + self.len() == 0 } - pub fn iter(&self) -> impl Iterator { - std::iter::once(&self.head).chain(self.others.iter()) + /// Returns an iterator over the equivalent orderings in this class. + pub fn iter(&self) -> impl Iterator { + self.orderings.iter() } + /// Returns how many equivalent orderings there are in this class. pub fn len(&self) -> usize { - self.others.len() + 1 - } - - pub fn is_empty(&self) -> bool { - self.len() == 0 + self.orderings.len() } -} -/// `LexOrdering` stores the lexicographical ordering for a schema. -/// OrderingEquivalentClass keeps track of different alternative orderings than can -/// describe the schema. -/// For instance, for the table below -/// |a|b|c|d| -/// |1|4|3|1| -/// |2|3|3|2| -/// |3|1|2|2| -/// |3|2|1|3| -/// both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the ordering of the table. -/// For this case, we say that `vec![a ASC, b ASC]`, and `vec![c DESC, d ASC]` are ordering equivalent. -pub type OrderingEquivalentClass = EquivalentClass; - -/// Update each expression in `ordering` with alias expressions. Assume -/// `ordering` is `a ASC, b ASC` and `c` is alias of `b`. Then, the result -/// will be `a ASC, c ASC`. -fn update_with_alias( - mut ordering: LexOrdering, - oeq_alias_map: &[(Column, Column)], -) -> LexOrdering { - for (source_col, target_col) in oeq_alias_map { - let source_col: Arc = Arc::new(source_col.clone()); - // Replace invalidated columns with its alias in the ordering expression. - let target_col: Arc = Arc::new(target_col.clone()); - for item in ordering.iter_mut() { - if item.expr.eq(&source_col) { - // Change the corresponding entry with alias expression - item.expr = target_col.clone(); - } - } + /// Extend this ordering equivalence class with the `other` class. + pub fn extend(&mut self, other: Self) { + self.orderings.extend(other.orderings); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); } - ordering -} -impl OrderingEquivalentClass { - /// This function updates ordering equivalences with alias information. - /// For instance, assume columns `a` and `b` are aliases (a as b), and - /// orderings `a ASC` and `c DESC` are equivalent. Here, we replace column - /// `a` with `b` in ordering equivalence expressions. After this function, - /// `a ASC`, `c DESC` will be converted to the `b ASC`, `c DESC`. - fn update_with_aliases( + /// Adds new orderings into this ordering equivalence class. + pub fn add_new_orderings( &mut self, - oeq_alias_map: &[(Column, Column)], - fields: &Fields, + orderings: impl IntoIterator, ) { - let is_head_invalid = self.head.iter().any(|sort_expr| { - collect_columns(&sort_expr.expr) - .iter() - .any(|col| is_column_invalid_in_new_schema(col, fields)) - }); - // If head is invalidated, update head with alias expressions - if is_head_invalid { - self.head = update_with_alias(self.head.clone(), oeq_alias_map); - } else { - let new_oeq_expr = update_with_alias(self.head.clone(), oeq_alias_map); - self.insert(new_oeq_expr); - } - for ordering in self.others.clone().into_iter() { - self.insert(update_with_alias(ordering, oeq_alias_map)); + self.orderings.extend(orderings); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); + } + + /// Removes redundant orderings from this equivalence class. + /// For instance, If we already have the ordering [a ASC, b ASC, c DESC], + /// then there is no need to keep ordering [a ASC, b ASC] in the state. + fn remove_redundant_entries(&mut self) { + let mut idx = 0; + while idx < self.orderings.len() { + let mut removal = false; + for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { + if let Some(right_finer) = finer_side(ordering, &self.orderings[idx]) { + if right_finer { + self.orderings.swap(ordering_idx, idx); + } + removal = true; + break; + } + } + if removal { + self.orderings.swap_remove(idx); + } else { + idx += 1; + } } } - /// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. - pub fn add_offset(&self, offset: usize) -> OrderingEquivalentClass { - let head = add_offset_to_lex_ordering(self.head(), offset); - let others = self - .others() - .iter() - .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) - .collect::>(); - OrderingEquivalentClass::new(head, others) + /// Gets the first ordering entry in this ordering equivalence class. + /// This is one of the many valid orderings (if there are multiple). + pub fn output_ordering(&self) -> Option { + self.orderings.first().cloned() } - /// This function normalizes `OrderingEquivalenceProperties` according to `eq_properties`. - /// More explicitly, it makes sure that expressions in `oeq_class` are head entries - /// in `eq_properties`, replacing any non-head entries with head entries if necessary. - pub fn normalize_with_equivalence_properties( - &self, - eq_properties: &EquivalenceProperties, - ) -> OrderingEquivalentClass { - let head = eq_properties.normalize_sort_exprs(self.head()); - - let others = self - .others() - .iter() - .map(|other| eq_properties.normalize_sort_exprs(other)) - .collect(); + // Append orderings in `other` to all existing orderings in this equivalence + // class. + pub fn join_suffix(mut self, other: &Self) -> Self { + for ordering in other.iter() { + for idx in 0..self.orderings.len() { + self.orderings[idx].extend(ordering.iter().cloned()); + } + } + self + } - EquivalentClass::new(head, others) + /// Adds `offset` value to the index of each expression inside this + /// ordering equivalence class. + pub fn add_offset(&mut self, offset: usize) { + for ordering in self.orderings.iter_mut() { + for sort_expr in ordering { + sort_expr.expr = add_offset_to_expr(sort_expr.expr.clone(), offset); + } + } } - /// Prefix with existing ordering. - pub fn prefix_ordering_equivalent_class_with_existing_ordering( - &self, - existing_ordering: &[PhysicalSortExpr], - eq_properties: &EquivalenceProperties, - ) -> OrderingEquivalentClass { - let existing_ordering = eq_properties.normalize_sort_exprs(existing_ordering); - let normalized_head = eq_properties.normalize_sort_exprs(self.head()); - let updated_head = merge_vectors(&existing_ordering, &normalized_head); - let updated_others = self - .others() - .iter() - .map(|ordering| { - let normalized_ordering = eq_properties.normalize_sort_exprs(ordering); - merge_vectors(&existing_ordering, &normalized_ordering) - }) - .collect(); - OrderingEquivalentClass::new(updated_head, updated_others) + /// Gets sort options associated with this expression if it is a leading + /// ordering expression. Otherwise, returns `None`. + fn get_options(&self, expr: &Arc) -> Option { + for ordering in self.iter() { + let leading_ordering = &ordering[0]; + if leading_ordering.expr.eq(expr) { + return Some(leading_ordering.options); + } + } + None } } -/// This is a builder object facilitating incremental construction -/// for ordering equivalences. -pub struct OrderingEquivalenceBuilder { - eq_properties: EquivalenceProperties, - ordering_eq_properties: OrderingEquivalenceProperties, - existing_ordering: Vec, +/// Adds the `offset` value to `Column` indices inside `expr`. This function is +/// generally used during the update of the right table schema in join operations. +pub fn add_offset_to_expr( + expr: Arc, + offset: usize, +) -> Arc { + expr.transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( + col.name(), + offset + col.index(), + )))), + None => Ok(Transformed::No(e)), + }) + .unwrap() + // Note that we can safely unwrap here since our transform always returns + // an `Ok` value. +} + +/// Returns `true` if the ordering `rhs` is strictly finer than the ordering `rhs`, +/// `false` if the ordering `lhs` is at least as fine as the ordering `lhs`, and +/// `None` otherwise (i.e. when given orderings are incomparable). +fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { + let all_equal = lhs.iter().zip(rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)); + all_equal.then_some(lhs.len() < rhs.len()) +} + +/// A `EquivalenceProperties` object stores useful information related to a schema. +/// Currently, it keeps track of: +/// - Equivalent expressions, e.g expressions that have same value. +/// - Valid sort expressions (orderings) for the schema. +/// - Constants expressions (e.g expressions that are known to have constant values). +/// +/// Consider table below: +/// +/// ```text +/// ┌-------┐ +/// | a | b | +/// |---|---| +/// | 1 | 9 | +/// | 2 | 8 | +/// | 3 | 7 | +/// | 5 | 5 | +/// └---┴---┘ +/// ``` +/// +/// where both `a ASC` and `b DESC` can describe the table ordering. With +/// `EquivalenceProperties`, we can keep track of these different valid sort +/// expressions and treat `a ASC` and `b DESC` on an equal footing. +/// +/// Similarly, consider the table below: +/// +/// ```text +/// ┌-------┐ +/// | a | b | +/// |---|---| +/// | 1 | 1 | +/// | 2 | 2 | +/// | 3 | 3 | +/// | 5 | 5 | +/// └---┴---┘ +/// ``` +/// +/// where columns `a` and `b` always have the same value. We keep track of such +/// equivalences inside this object. With this information, we can optimize +/// things like partitioning. For example, if the partition requirement is +/// `Hash(a)` and output partitioning is `Hash(b)`, then we can deduce that +/// the existing partitioning satisfies the requirement. +#[derive(Debug, Clone)] +pub struct EquivalenceProperties { + /// Collection of equivalence classes that store expressions with the same + /// value. + eq_group: EquivalenceGroup, + /// Equivalent sort expressions for this table. + oeq_class: OrderingEquivalenceClass, + /// Expressions whose values are constant throughout the table. + /// TODO: We do not need to track constants separately, they can be tracked + /// inside `eq_groups` as `Literal` expressions. + constants: Vec>, + /// Schema associated with this object. schema: SchemaRef, } -impl OrderingEquivalenceBuilder { +impl EquivalenceProperties { + /// Creates an empty `EquivalenceProperties` object. pub fn new(schema: SchemaRef) -> Self { - let eq_properties = EquivalenceProperties::new(schema.clone()); - let ordering_eq_properties = OrderingEquivalenceProperties::new(schema.clone()); Self { - eq_properties, - ordering_eq_properties, - existing_ordering: vec![], + eq_group: EquivalenceGroup::empty(), + oeq_class: OrderingEquivalenceClass::empty(), + constants: vec![], schema, } } - pub fn extend( - mut self, - new_ordering_eq_properties: OrderingEquivalenceProperties, - ) -> Self { - self.ordering_eq_properties - .extend(new_ordering_eq_properties.oeq_class().cloned()); - self + /// Creates a new `EquivalenceProperties` object with the given orderings. + pub fn new_with_orderings(schema: SchemaRef, orderings: &[LexOrdering]) -> Self { + Self { + eq_group: EquivalenceGroup::empty(), + oeq_class: OrderingEquivalenceClass::new(orderings.to_vec()), + constants: vec![], + schema, + } } - pub fn with_existing_ordering( - mut self, - existing_ordering: Option>, - ) -> Self { - if let Some(existing_ordering) = existing_ordering { - self.existing_ordering = existing_ordering; - } - self + /// Returns the associated schema. + pub fn schema(&self) -> &SchemaRef { + &self.schema } - pub fn with_equivalences(mut self, new_eq_properties: EquivalenceProperties) -> Self { - self.eq_properties = new_eq_properties; - self + /// Returns a reference to the ordering equivalence class within. + pub fn oeq_class(&self) -> &OrderingEquivalenceClass { + &self.oeq_class } - pub fn add_equal_conditions( - &mut self, - new_equivalent_ordering: Vec, - ) { - let mut normalized_out_ordering = vec![]; - for item in &self.existing_ordering { - // To account for ordering equivalences, first normalize the expression: - let normalized = self.eq_properties.normalize_expr(item.expr.clone()); - normalized_out_ordering.push(PhysicalSortExpr { - expr: normalized, - options: item.options, - }); - } - // If there is an existing ordering, add new ordering as an equivalence: - if !normalized_out_ordering.is_empty() { - self.ordering_eq_properties.add_equal_conditions(( - &normalized_out_ordering, - &new_equivalent_ordering, - )); - } + /// Returns a reference to the equivalence group within. + pub fn eq_group(&self) -> &EquivalenceGroup { + &self.eq_group } - /// Return a reference to the schema with which this builder was constructed with - pub fn schema(&self) -> &SchemaRef { - &self.schema + /// Returns the normalized version of the ordering equivalence class within. + /// Normalization removes constants and duplicates as well as standardizing + /// expressions according to the equivalence group within. + pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { + OrderingEquivalenceClass::new( + self.oeq_class + .iter() + .map(|ordering| self.normalize_sort_exprs(ordering)) + .collect(), + ) } - /// Return a reference to the existing ordering - pub fn existing_ordering(&self) -> &LexOrdering { - &self.existing_ordering + /// Extends this `EquivalenceProperties` with the `other` object. + pub fn extend(mut self, other: Self) -> Self { + self.eq_group.extend(other.eq_group); + self.oeq_class.extend(other.oeq_class); + self.add_constants(other.constants) } - pub fn build(self) -> OrderingEquivalenceProperties { - self.ordering_eq_properties + /// Clears (empties) the ordering equivalence class within this object. + /// Call this method when existing orderings are invalidated. + pub fn clear_orderings(&mut self) { + self.oeq_class.clear(); } -} - -/// Checks whether column is still valid after projection. -fn is_column_invalid_in_new_schema(column: &Column, fields: &Fields) -> bool { - let idx = column.index(); - idx >= fields.len() || fields[idx].name() != column.name() -} -/// Gets first aliased version of `col` found in `alias_map`. -fn get_alias_column( - col: &Column, - alias_map: &HashMap>, -) -> Option { - alias_map - .iter() - .find_map(|(column, columns)| column.eq(col).then(|| columns[0].clone())) -} - -/// This function applies the given projection to the given equivalence -/// properties to compute the resulting (projected) equivalence properties; e.g. -/// 1) Adding an alias, which can introduce additional equivalence properties, -/// as in Projection(a, a as a1, a as a2). -/// 2) Truncate the [`EquivalentClass`]es that are not in the output schema. -pub fn project_equivalence_properties( - input_eq: EquivalenceProperties, - alias_map: &HashMap>, - output_eq: &mut EquivalenceProperties, -) { - // Get schema and fields of projection output - let schema = output_eq.schema(); - let fields = schema.fields(); - - let mut eq_classes = input_eq.classes().to_vec(); - for (column, columns) in alias_map { - let mut find_match = false; - for class in eq_classes.iter_mut() { - // If `self.head` is invalidated in the new schema, update head - // with this change `self.head` is not randomly assigned by one of the entries from `self.others` - if is_column_invalid_in_new_schema(&class.head, fields) { - if let Some(alias_col) = get_alias_column(&class.head, alias_map) { - class.head = alias_col; - } - } - if class.contains(column) { - for col in columns { - class.insert(col.clone()); - } - find_match = true; - break; - } - } - if !find_match { - eq_classes.push(EquivalentClass::new(column.clone(), columns.clone())); - } + /// Extends this `EquivalenceProperties` by adding the orderings inside the + /// ordering equivalence class `other`. + pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { + self.oeq_class.extend(other); } - // Prune columns that are no longer in the schema from equivalences. - for class in eq_classes.iter_mut() { - let columns_to_remove = class - .iter() - .filter(|column| is_column_invalid_in_new_schema(column, fields)) - .cloned() - .collect::>(); - for column in columns_to_remove { - class.remove(&column); - } + /// Adds new orderings into the existing ordering equivalence class. + pub fn add_new_orderings( + &mut self, + orderings: impl IntoIterator, + ) { + self.oeq_class.add_new_orderings(orderings); } - eq_classes.retain(|props| { - props.len() > 1 - && - // A column should not give an equivalence with itself. - !(props.len() == 2 && props.head.eq(props.others().iter().next().unwrap())) - }); - - output_eq.extend(eq_classes); -} - -/// This function applies the given projection to the given ordering -/// equivalence properties to compute the resulting (projected) ordering -/// equivalence properties; e.g. -/// 1) Adding an alias, which can introduce additional ordering equivalence -/// properties, as in Projection(a, a as a1, a as a2) extends global ordering -/// of a to a1 and a2. -/// 2) Truncate the [`OrderingEquivalentClass`]es that are not in the output schema. -pub fn project_ordering_equivalence_properties( - input_eq: OrderingEquivalenceProperties, - columns_map: &HashMap>, - output_eq: &mut OrderingEquivalenceProperties, -) { - // Get schema and fields of projection output - let schema = output_eq.schema(); - let fields = schema.fields(); - - let oeq_class = input_eq.oeq_class(); - let mut oeq_class = if let Some(oeq_class) = oeq_class { - oeq_class.clone() - } else { - return; - }; - let mut oeq_alias_map = vec![]; - for (column, columns) in columns_map { - if is_column_invalid_in_new_schema(column, fields) { - oeq_alias_map.push((column.clone(), columns[0].clone())); - } - } - oeq_class.update_with_aliases(&oeq_alias_map, fields); - - // Prune columns that no longer is in the schema from from the OrderingEquivalenceProperties. - let sort_exprs_to_remove = oeq_class - .iter() - .filter(|sort_exprs| { - sort_exprs.iter().any(|sort_expr| { - let cols_in_expr = collect_columns(&sort_expr.expr); - // If any one of the columns, used in Expression is invalid, remove expression - // from ordering equivalences - cols_in_expr - .iter() - .any(|col| is_column_invalid_in_new_schema(col, fields)) - }) - }) - .cloned() - .collect::>(); - for sort_exprs in sort_exprs_to_remove { - oeq_class.remove(&sort_exprs); + /// Incorporates the given equivalence group to into the existing + /// equivalence group within. + pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { + self.eq_group.extend(other_eq_group); } - if oeq_class.len() > 1 { - output_eq.extend(Some(oeq_class)); - } -} -/// Update `ordering` if it contains cast expression with target column -/// after projection, if there is no cast expression among `ordering` expressions, -/// returns `None`. -fn update_with_cast_exprs( - cast_exprs: &[(CastExpr, Column)], - mut ordering: LexOrdering, -) -> Option { - let mut is_changed = false; - for sort_expr in ordering.iter_mut() { - for (cast_expr, target_col) in cast_exprs.iter() { - if sort_expr.expr.eq(cast_expr.expr()) { - sort_expr.expr = Arc::new(target_col.clone()) as _; - is_changed = true; - } - } + /// Adds a new equality condition into the existing equivalence group. + /// If the given equality defines a new equivalence class, adds this new + /// equivalence class to the equivalence group. + pub fn add_equal_conditions( + &mut self, + left: &Arc, + right: &Arc, + ) { + self.eq_group.add_equal_conditions(left, right); } - is_changed.then_some(ordering) -} -/// Update cast expressions inside ordering equivalence -/// properties with its target column after projection -pub fn update_ordering_equivalence_with_cast( - cast_exprs: &[(CastExpr, Column)], - input_oeq: &mut OrderingEquivalenceProperties, -) { - if let Some(cls) = &mut input_oeq.oeq_class { - for ordering in - std::iter::once(cls.head().clone()).chain(cls.others().clone().into_iter()) - { - if let Some(updated_ordering) = update_with_cast_exprs(cast_exprs, ordering) { - cls.insert(updated_ordering); + /// Track/register physical expressions with constant values. + pub fn add_constants( + mut self, + constants: impl IntoIterator>, + ) -> Self { + for expr in self.eq_group.normalize_exprs(constants) { + if !physical_exprs_contains(&self.constants, &expr) { + self.constants.push(expr); } } + self } -} - -/// Retrieves the ordering equivalence properties for a given schema and output ordering. -pub fn ordering_equivalence_properties_helper( - schema: SchemaRef, - eq_orderings: &[LexOrdering], -) -> OrderingEquivalenceProperties { - let mut oep = OrderingEquivalenceProperties::new(schema); - let first_ordering = if let Some(first) = eq_orderings.first() { - first - } else { - // Return an empty OrderingEquivalenceProperties: - return oep; - }; - // First entry among eq_orderings is the head, skip it: - for ordering in eq_orderings.iter().skip(1) { - if !ordering.is_empty() { - oep.add_equal_conditions((first_ordering, ordering)) - } - } - oep -} - -/// This function constructs a duplicate-free vector by filtering out duplicate -/// entries inside the given vector `input`. -fn collapse_vec(input: Vec) -> Vec { - let mut output = vec![]; - for item in input { - if !output.contains(&item) { - output.push(item); - } - } - output -} -/// This function constructs a duplicate-free `LexOrderingReq` by filtering out duplicate -/// entries that have same physical expression inside the given vector `input`. -/// `vec![a Some(Asc), a Some(Desc)]` is collapsed to the `vec![a Some(Asc)]`. Since -/// when same expression is already seen before, following expressions are redundant. -fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { - let mut output = vec![]; - for item in input { - if !lex_req_contains(&output, &item) { - output.push(item); - } + /// Updates the ordering equivalence group within assuming that the table + /// is re-sorted according to the argument `sort_exprs`. Note that constants + /// and equivalence classes are unchanged as they are unaffected by a re-sort. + pub fn with_reorder(mut self, sort_exprs: Vec) -> Self { + // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. + self.oeq_class = OrderingEquivalenceClass::new(vec![sort_exprs]); + self } - output -} -/// Check whether `sort_req.expr` is among the expressions of `lex_req`. -fn lex_req_contains( - lex_req: &[PhysicalSortRequirement], - sort_req: &PhysicalSortRequirement, -) -> bool { - for constant in lex_req { - if constant.expr.eq(&sort_req.expr) { + /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the + /// equivalence group and the ordering equivalence class within. + /// + /// Assume that `self.eq_group` states column `a` and `b` are aliases. + /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` + /// are equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_exprs` argument were `vec![b ASC, c ASC, a ASC]`, then this + /// function would return `vec![a ASC, c ASC]`. Internally, it would first + /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result + /// after deduplication. + fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + // Convert sort expressions to sort requirements: + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + // Normalize the requirements: + let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); + // Convert sort requirements back to sort expressions: + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) + } + + /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the + /// equivalence group and the ordering equivalence class within. It works by: + /// - Removing expressions that have a constant value from the given requirement. + /// - Replacing sections that belong to some equivalence class in the equivalence + /// group with the first entry in the matching equivalence class. + /// + /// Assume that `self.eq_group` states column `a` and `b` are aliases. + /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` + /// are equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_reqs` argument were `vec![b ASC, c ASC, a ASC]`, then this + /// function would return `vec![a ASC, c ASC]`. Internally, it would first + /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result + /// after deduplication. + fn normalize_sort_requirements( + &self, + sort_reqs: LexRequirementRef, + ) -> LexRequirement { + let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); + let constants_normalized = self.eq_group.normalize_exprs(self.constants.clone()); + // Prune redundant sections in the requirement: + collapse_lex_req( + normalized_sort_reqs + .iter() + .filter(|&order| { + !physical_exprs_contains(&constants_normalized, &order.expr) + }) + .cloned() + .collect(), + ) + } + + /// Checks whether the given ordering is satisfied by any of the existing + /// orderings. + pub fn ordering_satisfy(&self, given: LexOrderingRef) -> bool { + // Convert the given sort expressions to sort requirements: + let sort_requirements = PhysicalSortRequirement::from_sort_exprs(given.iter()); + self.ordering_satisfy_requirement(&sort_requirements) + } + + /// Checks whether the given sort requirements are satisfied by any of the + /// existing orderings. + pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + // First, standardize the given requirement: + let normalized_reqs = self.normalize_sort_requirements(reqs); + if normalized_reqs.is_empty() { + // Requirements are tautologically satisfied if empty. return true; } - } - false -} - -/// This function simplifies lexicographical ordering requirement -/// inside `input` by removing postfix lexicographical requirements -/// that satisfy global ordering (occurs inside the ordering equivalent class) -fn simplify_lex_req( - input: LexOrderingReq, - oeq_class: &OrderingEquivalentClass, -) -> LexOrderingReq { - let mut section = &input[..]; - loop { - let n_prune = prune_last_n_that_is_in_oeq(section, oeq_class); - // Cannot prune entries from the end of requirement - if n_prune == 0 { - break; + let mut indices = HashSet::new(); + for ordering in self.normalized_oeq_class().iter() { + let match_indices = ordering + .iter() + .map(|sort_expr| { + normalized_reqs + .iter() + .position(|sort_req| sort_expr.satisfy(sort_req, &self.schema)) + }) + .collect::>(); + // Find the largest contiguous increasing sequence starting from the first index: + if let Some(&Some(first)) = match_indices.first() { + indices.insert(first); + let mut iter = match_indices.windows(2); + while let Some([Some(current), Some(next)]) = iter.next() { + if next > current { + indices.insert(*next); + } else { + break; + } + } + } } - section = §ion[0..section.len() - n_prune]; + indices.len() == normalized_reqs.len() } - if section.is_empty() { - PhysicalSortRequirement::from_sort_exprs(oeq_class.head()) - } else { - section.to_vec() - } -} -/// Determines how many entries from the end can be deleted. -/// Last n entry satisfies global ordering, hence having them -/// as postfix in the lexicographical requirement is unnecessary. -/// Assume requirement is [a ASC, b ASC, c ASC], also assume that -/// existing ordering is [c ASC, d ASC]. In this case, since [c ASC] -/// is satisfied by the existing ordering (e.g corresponding section is global ordering), -/// [c ASC] can be pruned from the requirement: [a ASC, b ASC, c ASC]. In this case, -/// this function will return 1, to indicate last element can be removed from the requirement -fn prune_last_n_that_is_in_oeq( - input: &[PhysicalSortRequirement], - oeq_class: &OrderingEquivalentClass, -) -> usize { - let input_len = input.len(); - for ordering in std::iter::once(oeq_class.head()).chain(oeq_class.others().iter()) { - let mut search_range = std::cmp::min(ordering.len(), input_len); - while search_range > 0 { - let req_section = &input[input_len - search_range..]; - // let given_section = &ordering[0..search_range]; - if req_satisfied(ordering, req_section) { - return search_range; + /// Checks whether the `given`` sort requirements are equal or more specific + /// than the `reference` sort requirements. + pub fn requirements_compatible( + &self, + given: LexRequirementRef, + reference: LexRequirementRef, + ) -> bool { + let normalized_given = self.normalize_sort_requirements(given); + let normalized_reference = self.normalize_sort_requirements(reference); + + (normalized_reference.len() <= normalized_given.len()) + && normalized_reference + .into_iter() + .zip(normalized_given) + .all(|(reference, given)| given.compatible(&reference)) + } + + /// Returns the finer ordering among the orderings `lhs` and `rhs`, breaking + /// any ties by choosing `lhs`. + /// + /// The finer ordering is the ordering that satisfies both of the orderings. + /// If the orderings are incomparable, returns `None`. + /// + /// For example, the finer ordering among `[a ASC]` and `[a ASC, b ASC]` is + /// the latter. + pub fn get_finer_ordering( + &self, + lhs: LexOrderingRef, + rhs: LexOrderingRef, + ) -> Option { + // Convert the given sort expressions to sort requirements: + let lhs = PhysicalSortRequirement::from_sort_exprs(lhs); + let rhs = PhysicalSortRequirement::from_sort_exprs(rhs); + let finer = self.get_finer_requirement(&lhs, &rhs); + // Convert the chosen sort requirements back to sort expressions: + finer.map(PhysicalSortRequirement::to_sort_exprs) + } + + /// Returns the finer ordering among the requirements `lhs` and `rhs`, + /// breaking any ties by choosing `lhs`. + /// + /// The finer requirements are the ones that satisfy both of the given + /// requirements. If the requirements are incomparable, returns `None`. + /// + /// For example, the finer requirements among `[a ASC]` and `[a ASC, b ASC]` + /// is the latter. + pub fn get_finer_requirement( + &self, + req1: LexRequirementRef, + req2: LexRequirementRef, + ) -> Option { + let mut lhs = self.normalize_sort_requirements(req1); + let mut rhs = self.normalize_sort_requirements(req2); + lhs.iter_mut() + .zip(rhs.iter_mut()) + .all(|(lhs, rhs)| { + lhs.expr.eq(&rhs.expr) + && match (lhs.options, rhs.options) { + (Some(lhs_opt), Some(rhs_opt)) => lhs_opt == rhs_opt, + (Some(options), None) => { + rhs.options = Some(options); + true + } + (None, Some(options)) => { + lhs.options = Some(options); + true + } + (None, None) => true, + } + }) + .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) + } + + /// Calculates the "meet" of the given orderings (`lhs` and `rhs`). + /// The meet of a set of orderings is the finest ordering that is satisfied + /// by all the orderings in that set. For details, see: + /// + /// + /// + /// If there is no ordering that satisfies both `lhs` and `rhs`, returns + /// `None`. As an example, the meet of orderings `[a ASC]` and `[a ASC, b ASC]` + /// is `[a ASC]`. + pub fn get_meet_ordering( + &self, + lhs: LexOrderingRef, + rhs: LexOrderingRef, + ) -> Option { + let lhs = self.normalize_sort_exprs(lhs); + let rhs = self.normalize_sort_exprs(rhs); + let mut meet = vec![]; + for (lhs, rhs) in lhs.into_iter().zip(rhs.into_iter()) { + if lhs.eq(&rhs) { + meet.push(lhs); } else { - search_range -= 1; + break; } } + (!meet.is_empty()).then_some(meet) + } + + /// Projects argument `expr` according to `projection_mapping`, taking + /// equivalences into account. + /// + /// For example, assume that columns `a` and `c` are always equal, and that + /// `projection_mapping` encodes following mapping: + /// + /// ```text + /// a -> a1 + /// b -> b1 + /// ``` + /// + /// Then, this function projects `a + b` to `Some(a1 + b1)`, `c + b` to + /// `Some(a1 + b1)` and `d` to `None`, meaning that it cannot be projected. + pub fn project_expr( + &self, + expr: &Arc, + projection_mapping: &ProjectionMapping, + ) -> Option> { + self.eq_group.project_expr(projection_mapping, expr) } - 0 -} -/// Checks whether given section satisfies req. -fn req_satisfied(given: LexOrderingRef, req: &[PhysicalSortRequirement]) -> bool { - for (given, req) in izip!(given.iter(), req.iter()) { - let PhysicalSortRequirement { expr, options } = req; - if let Some(options) = options { - if options != &given.options || !expr.eq(&given.expr) { - return false; + /// Projects the equivalences within according to `projection_mapping` + /// and `output_schema`. + pub fn project( + &self, + projection_mapping: &ProjectionMapping, + output_schema: SchemaRef, + ) -> Self { + let mut projected_orderings = self + .oeq_class + .iter() + .filter_map(|order| self.eq_group.project_ordering(projection_mapping, order)) + .collect::>(); + for (source, target) in projection_mapping { + let expr_ordering = ExprOrdering::new(source.clone()) + .transform_up(&|expr| update_ordering(expr, self)) + .unwrap(); + if let SortProperties::Ordered(options) = expr_ordering.state { + // Push new ordering to the state. + projected_orderings.push(vec![PhysicalSortExpr { + expr: target.clone(), + options, + }]); } - } else if !expr.eq(&given.expr) { - return false; - } - } - true -} - -/// Combine equivalence properties of the given join inputs. -pub fn combine_join_equivalence_properties( - join_type: JoinType, - left_properties: EquivalenceProperties, - right_properties: EquivalenceProperties, - left_columns_len: usize, - on: &[(Column, Column)], - schema: SchemaRef, -) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(schema); - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - new_properties.extend(left_properties.classes().to_vec()); - let new_right_properties = right_properties - .classes() - .iter() - .map(|prop| { - let new_head = Column::new( - prop.head().name(), - left_columns_len + prop.head().index(), - ); - let new_others = prop - .others() - .iter() - .map(|col| { - Column::new(col.name(), left_columns_len + col.index()) - }) - .collect::>(); - EquivalentClass::new(new_head, new_others) - }) - .collect::>(); - - new_properties.extend(new_right_properties); - } - JoinType::LeftSemi | JoinType::LeftAnti => { - new_properties.extend(left_properties.classes().to_vec()) } - JoinType::RightSemi | JoinType::RightAnti => { - new_properties.extend(right_properties.classes().to_vec()) + Self { + eq_group: self.eq_group.project(projection_mapping), + oeq_class: OrderingEquivalenceClass::new(projected_orderings), + constants: vec![], + schema: output_schema, } } - if join_type == JoinType::Inner { - on.iter().for_each(|(column1, column2)| { - let new_column2 = - Column::new(column2.name(), left_columns_len + column2.index()); - new_properties.add_equal_conditions((column1, &new_column2)) - }) - } - new_properties -} - -/// Calculate equivalence properties for the given cross join operation. -pub fn cross_join_equivalence_properties( - left_properties: EquivalenceProperties, - right_properties: EquivalenceProperties, - left_columns_len: usize, - schema: SchemaRef, -) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(schema); - new_properties.extend(left_properties.classes().to_vec()); - let new_right_properties = right_properties - .classes() - .iter() - .map(|prop| { - let new_head = - Column::new(prop.head().name(), left_columns_len + prop.head().index()); - let new_others = prop - .others() - .iter() - .map(|col| Column::new(col.name(), left_columns_len + col.index())) - .collect::>(); - EquivalentClass::new(new_head, new_others) - }) - .collect::>(); - new_properties.extend(new_right_properties); - new_properties -} - -/// Update right table ordering equivalences so that: -/// - They point to valid indices at the output of the join schema, and -/// - They are normalized with respect to equivalence columns. -/// -/// To do so, we increment column indices by the size of the left table when -/// join schema consists of a combination of left and right schema (Inner, -/// Left, Full, Right joins). Then, we normalize the sort expressions of -/// ordering equivalences one by one. We make sure that each expression in the -/// ordering equivalence is either: -/// - The head of the one of the equivalent classes, or -/// - Doesn't have an equivalent column. -/// -/// This way; once we normalize an expression according to equivalence properties, -/// it can thereafter safely be used for ordering equivalence normalization. -fn get_updated_right_ordering_equivalent_class( - join_type: &JoinType, - right_oeq_class: &OrderingEquivalentClass, - left_columns_len: usize, - join_eq_properties: &EquivalenceProperties, -) -> OrderingEquivalentClass { - match join_type { - // In these modes, indices of the right schema should be offset by - // the left table size. - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - let right_oeq_class = right_oeq_class.add_offset(left_columns_len); - return right_oeq_class - .normalize_with_equivalence_properties(join_eq_properties); + /// Returns the longest (potentially partial) permutation satisfying the + /// existing ordering. For example, if we have the equivalent orderings + /// `[a ASC, b ASC]` and `[c DESC]`, with `exprs` containing `[c, b, a, d]`, + /// then this function returns `([a ASC, b ASC, c DESC], [2, 1, 0])`. + /// This means that the specification `[a ASC, b ASC, c DESC]` is satisfied + /// by the existing ordering, and `[a, b, c]` resides at indices: `2, 1, 0` + /// inside the argument `exprs` (respectively). For the mathematical + /// definition of "partial permutation", see: + /// + /// + pub fn find_longest_permutation( + &self, + exprs: &[Arc], + ) -> (LexOrdering, Vec) { + let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); + // Use a map to associate expression indices with sort options: + let mut ordered_exprs = IndexMap::::new(); + for ordering in self.normalized_oeq_class().iter() { + for sort_expr in ordering { + if let Some(idx) = normalized_exprs + .iter() + .position(|expr| sort_expr.expr.eq(expr)) + { + if let Entry::Vacant(e) = ordered_exprs.entry(idx) { + e.insert(sort_expr.options); + } + } else { + // We only consider expressions that correspond to a prefix + // of one of the equivalent orderings we have. + break; + } + } } - _ => {} - }; - right_oeq_class.normalize_with_equivalence_properties(join_eq_properties) + // Construct the lexicographical ordering according to the permutation: + ordered_exprs + .into_iter() + .map(|(idx, options)| { + ( + PhysicalSortExpr { + expr: exprs[idx].clone(), + options, + }, + idx, + ) + }) + .unzip() + } } /// Calculate ordering equivalence properties for the given join operation. -pub fn combine_join_ordering_equivalence_properties( +pub fn join_equivalence_properties( + left: EquivalenceProperties, + right: EquivalenceProperties, join_type: &JoinType, - left_oeq_properties: &OrderingEquivalenceProperties, - right_oeq_properties: &OrderingEquivalenceProperties, - schema: SchemaRef, + join_schema: SchemaRef, maintains_input_order: &[bool], probe_side: Option, - join_eq_properties: EquivalenceProperties, -) -> OrderingEquivalenceProperties { - let mut new_properties = OrderingEquivalenceProperties::new(schema); - let left_columns_len = left_oeq_properties.schema().fields().len(); - // All joins have 2 children - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; - match (left_maintains, right_maintains) { - (true, true) => { - unreachable!("Cannot maintain ordering of both sides"); - } - (true, false) => { - // In this special case, right side ordering can be prefixed with left side ordering. - if let ( - Some(JoinSide::Left), - JoinType::Inner, - Some(left_oeq_class), - Some(right_oeq_class), - ) = ( - probe_side, - join_type, - left_oeq_properties.oeq_class(), - right_oeq_properties.oeq_class(), - ) { - let updated_right_oeq = get_updated_right_ordering_equivalent_class( + on: &[(Column, Column)], +) -> EquivalenceProperties { + let left_size = left.schema.fields.len(); + let mut result = EquivalenceProperties::new(join_schema); + result.add_equivalence_group(left.eq_group().join( + right.eq_group(), + join_type, + left_size, + on, + )); + + let left_oeq_class = left.oeq_class; + let mut right_oeq_class = right.oeq_class; + match maintains_input_order { + [true, false] => { + // In this special case, right side ordering can be prefixed with + // the left side ordering. + if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { + updated_right_ordering_equivalence_class( + &mut right_oeq_class, join_type, - right_oeq_class, - left_columns_len, - &join_eq_properties, + left_size, ); - // Right side ordering equivalence properties should be prepended with - // those of the left side while constructing output ordering equivalence - // properties since stream side is the left side. + // Right side ordering equivalence properties should be prepended + // with those of the left side while constructing output ordering + // equivalence properties since stream side is the left side. // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let mut orderings = vec![]; - for left_ordering in left_oeq_class.iter() { - for right_ordering in updated_right_oeq.iter() { - let mut ordering = left_ordering.to_vec(); - ordering.extend(right_ordering.to_vec()); - let ordering_normalized = - join_eq_properties.normalize_sort_exprs(&ordering); - orderings.push(ordering_normalized); - } - } - if !orderings.is_empty() { - let head = orderings.swap_remove(0); - let new_oeq_class = OrderingEquivalentClass::new(head, orderings); - new_properties.extend(Some(new_oeq_class)); - } + // For example, if the right side ordering equivalences contain + // `b ASC`, and the left side ordering equivalences contain `a ASC`, + // then we should add `a ASC, b ASC` to the ordering equivalences + // of the join output. + let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); + result.add_ordering_equivalence_class(out_oeq_class); } else { - new_properties.extend(left_oeq_properties.oeq_class().cloned()); + result.add_ordering_equivalence_class(left_oeq_class); } } - (false, true) => { - let updated_right_oeq = - right_oeq_properties.oeq_class().map(|right_oeq_class| { - get_updated_right_ordering_equivalent_class( - join_type, - right_oeq_class, - left_columns_len, - &join_eq_properties, - ) - }); - // In this special case, left side ordering can be prefixed with right side ordering. - if let ( - Some(JoinSide::Right), - JoinType::Inner, - Some(left_oeq_class), - Some(right_oeg_class), - ) = ( - probe_side, + [false, true] => { + updated_right_ordering_equivalence_class( + &mut right_oeq_class, join_type, - left_oeq_properties.oeq_class(), - &updated_right_oeq, - ) { - // Left side ordering equivalence properties should be prepended with - // those of the right side while constructing output ordering equivalence - // properties since stream side is the right side. + left_size, + ); + // In this special case, left side ordering can be prefixed with + // the right side ordering. + if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { + // Left side ordering equivalence properties should be prepended + // with those of the right side while constructing output ordering + // equivalence properties since stream side is the right side. // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let mut orderings = vec![]; - for right_ordering in right_oeg_class.iter() { - for left_ordering in left_oeq_class.iter() { - let mut ordering = right_ordering.to_vec(); - ordering.extend(left_ordering.to_vec()); - let ordering_normalized = - join_eq_properties.normalize_sort_exprs(&ordering); - orderings.push(ordering_normalized); - } - } - if !orderings.is_empty() { - let head = orderings.swap_remove(0); - let new_oeq_class = OrderingEquivalentClass::new(head, orderings); - new_properties.extend(Some(new_oeq_class)); - } + // For example, if the left side ordering equivalences contain + // `a ASC`, and the right side ordering equivalences contain `b ASC`, + // then we should add `b ASC, a ASC` to the ordering equivalences + // of the join output. + let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); + result.add_ordering_equivalence_class(out_oeq_class); } else { - new_properties.extend(updated_right_oeq); + result.add_ordering_equivalence_class(right_oeq_class); } } - (false, false) => {} + [false, false] => {} + [true, true] => unreachable!("Cannot maintain ordering of both sides"), + _ => unreachable!("Join operators can not have more than two children"), } - new_properties -} - -/// This function searches for the slice `section` inside the slice `given`. -/// It returns each range where `section` is compatible with the corresponding -/// slice in `given`. -fn get_compatible_ranges( - given: &[PhysicalSortRequirement], - section: &[PhysicalSortRequirement], -) -> Vec> { - let n_section = section.len(); - let n_end = if given.len() >= n_section { - given.len() - n_section + 1 - } else { - 0 - }; - (0..n_end) - .filter_map(|idx| { - let end = idx + n_section; - given[idx..end] - .iter() - .zip(section) - .all(|(req, given)| given.compatible(req)) - .then_some(Range { start: idx, end }) - }) - .collect() + result } -/// It is similar to contains method of vector. -/// Finds whether `expr` is among `physical_exprs`. -pub fn physical_exprs_contains( - physical_exprs: &[Arc], - expr: &Arc, -) -> bool { - physical_exprs - .iter() - .any(|physical_expr| physical_expr.eq(expr)) -} - -/// Remove ordering requirements that have constant value -fn prune_sort_reqs_with_constants( - ordering: &[PhysicalSortRequirement], - constants: &[Arc], -) -> Vec { - ordering - .iter() - .filter(|&order| !physical_exprs_contains(constants, &order.expr)) - .cloned() - .collect() -} - -/// Adds the `offset` value to `Column` indices inside `expr`. This function is -/// generally used during the update of the right table schema in join operations. -pub fn add_offset_to_expr( - expr: Arc, - offset: usize, -) -> Arc { - expr.transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( - col.name(), - offset + col.index(), - )))), - None => Ok(Transformed::No(e)), - }) - .unwrap() - // Note that we can safely unwrap here since our transform always returns - // an `Ok` value. -} - -/// Adds the `offset` value to `Column` indices inside `sort_expr.expr`. -pub(crate) fn add_offset_to_sort_expr( - sort_expr: &PhysicalSortExpr, - offset: usize, -) -> PhysicalSortExpr { - PhysicalSortExpr { - expr: add_offset_to_expr(sort_expr.expr.clone(), offset), - options: sort_expr.options, +/// In the context of a join, update the right side `OrderingEquivalenceClass` +/// so that they point to valid indices in the join output schema. +/// +/// To do so, we increment column indices by the size of the left table when +/// join schema consists of a combination of the left and right schemas. This +/// is the case for `Inner`, `Left`, `Full` and `Right` joins. For other cases, +/// indices do not change. +fn updated_right_ordering_equivalence_class( + right_oeq_class: &mut OrderingEquivalenceClass, + join_type: &JoinType, + left_size: usize, +) { + if matches!( + join_type, + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right + ) { + right_oeq_class.add_offset(left_size); } } -/// Adds the `offset` value to `Column` indices for each `sort_expr.expr` -/// inside `sort_exprs`. -pub fn add_offset_to_lex_ordering( - sort_exprs: LexOrderingRef, - offset: usize, -) -> LexOrdering { - sort_exprs - .iter() - .map(|sort_expr| add_offset_to_sort_expr(sort_expr, offset)) - .collect() +/// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. +/// The node can either be a leaf node, or an intermediate node: +/// - If it is a leaf node, we directly find the order of the node by looking +/// at the given sort expression and equivalence properties if it is a `Column` +/// leaf, or we mark it as unordered. In the case of a `Literal` leaf, we mark +/// it as singleton so that it can cooperate with all ordered columns. +/// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` +/// and operator has its own rules on how to propagate the children orderings. +/// However, before we engage in recursion, we check whether this intermediate +/// node directly matches with the sort expression. If there is a match, the +/// sort expression emerges at that node immediately, discarding the recursive +/// result coming from its children. +fn update_ordering( + mut node: ExprOrdering, + eq_properties: &EquivalenceProperties, +) -> Result> { + if !node.expr.children().is_empty() { + // We have an intermediate (non-leaf) node, account for its children: + node.state = node.expr.get_ordering(&node.children_states); + Ok(Transformed::Yes(node)) + } else if node.expr.as_any().is::() { + // We have a Column, which is one of the two possible leaf node types: + let eq_group = &eq_properties.eq_group; + let normalized_expr = eq_group.normalize_expr(node.expr.clone()); + let oeq_class = &eq_properties.oeq_class; + if let Some(options) = oeq_class.get_options(&normalized_expr) { + node.state = SortProperties::Ordered(options); + Ok(Transformed::Yes(node)) + } else { + Ok(Transformed::No(node)) + } + } else { + // We have a Literal, which is the other possible leaf node type: + node.state = node.expr.get_ordering(&[]); + Ok(Transformed::Yes(node)) + } } #[cfg(test)] mod tests { + use std::sync::Arc; + use super::*; - use crate::expressions::Column; + use crate::expressions::{col, lit, BinaryExpr, Column}; + use crate::physical_expr::{physical_exprs_bag_equal, physical_exprs_equal}; + + use arrow::compute::{lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field, Schema}; + use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; + use arrow_schema::{Fields, SortOptions}; use datafusion_common::Result; + use datafusion_expr::Operator; + + use itertools::{izip, Itertools}; + use rand::rngs::StdRng; + use rand::seq::SliceRandom; + use rand::{Rng, SeedableRng}; + + // Generate a schema which consists of 8 columns (a, b, c, d, e, f, g, h) + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let g = Field::new("g", DataType::Int32, true); + let h = Field::new("h", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g, h])); + + Ok(schema) + } + + /// Construct a schema with following properties + /// Schema satisfies following orderings: + /// [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + /// and + /// Column [a=c] (e.g they are aliases). + fn create_test_params() -> Result<(SchemaRef, EquivalenceProperties)> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_g = &col("g", &test_schema)?; + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + eq_properties.add_equal_conditions(col_a, col_c); + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let orderings = vec![ + // [a ASC] + vec![(col_a, option_asc)], + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [e DESC, f ASC, g ASC] + vec![ + (col_e, option_desc), + (col_f, option_asc), + (col_g, option_asc), + ], + ]; + let orderings = convert_to_orderings(&orderings); + eq_properties.add_new_orderings(orderings); + Ok((test_schema, eq_properties)) + } + + // Generate a schema which consists of 6 columns (a, b, c, d, e, f) + fn create_test_schema_2() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); + + Ok(schema) + } + + /// Construct a schema with random ordering + /// among column a, b, c, d + /// where + /// Column [a=f] (e.g they are aliases). + /// Column e is constant. + fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperties)> { + let test_schema = create_test_schema_2()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_exprs = [col_a, col_b, col_c, col_d, col_e, col_f]; + + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + // Define a and f are aliases + eq_properties.add_equal_conditions(col_a, col_f); + // Column e has constant value. + eq_properties = eq_properties.add_constants([col_e.clone()]); + + // Randomly order columns for sorting + let mut rng = StdRng::seed_from_u64(seed); + let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted + + let options_asc = SortOptions { + descending: false, + nulls_first: false, + }; - use arrow_schema::SortOptions; - use std::sync::Arc; + while !remaining_exprs.is_empty() { + let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); + remaining_exprs.shuffle(&mut rng); + + let ordering = remaining_exprs + .drain(0..n_sort_expr) + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: options_asc, + }) + .collect(); + + eq_properties.add_new_orderings([ordering]); + } + + Ok((test_schema, eq_properties)) + } - fn convert_to_requirement( - in_data: &[(&Column, Option)], + // Convert each tuple to PhysicalSortRequirement + fn convert_to_sort_reqs( + in_data: &[(&Arc, Option)], ) -> Vec { in_data .iter() - .map(|(col, options)| { - PhysicalSortRequirement::new(Arc::new((*col).clone()) as _, *options) + .map(|(expr, options)| { + PhysicalSortRequirement::new((*expr).clone(), *options) + }) + .collect::>() + } + + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs( + in_data: &[(&Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, }) .collect::>() } + // Convert each inner tuple to PhysicalSortExpr + fn convert_to_orderings( + orderings: &[Vec<(&Arc, SortOptions)>], + ) -> Vec> { + orderings + .iter() + .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) + .collect() + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -1239,38 +1367,49 @@ mod tests { ])); let mut eq_properties = EquivalenceProperties::new(schema); - let new_condition = (&Column::new("a", 0), &Column::new("b", 1)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - - let new_condition = (&Column::new("b", 1), &Column::new("a", 0)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 2); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); - - let new_condition = (&Column::new("b", 1), &Column::new("c", 2)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 3); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); - assert!(eq_properties.classes()[0].contains(&Column::new("c", 2))); - - let new_condition = (&Column::new("x", 3), &Column::new("y", 4)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 2); - - let new_condition = (&Column::new("x", 3), &Column::new("a", 0)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 5); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); - assert!(eq_properties.classes()[0].contains(&Column::new("c", 2))); - assert!(eq_properties.classes()[0].contains(&Column::new("x", 3))); - assert!(eq_properties.classes()[0].contains(&Column::new("y", 4))); + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; + let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; + + // a and b are aliases + eq_properties.add_equal_conditions(&col_a_expr, &col_b_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + + // This new entry is redundant, size shouldn't increase + eq_properties.add_equal_conditions(&col_b_expr, &col_a_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 2); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + + // b and c are aliases. Exising equivalence class should expand, + // however there shouldn't be any new equivalence class + eq_properties.add_equal_conditions(&col_b_expr, &col_c_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 3); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + assert!(physical_exprs_contains(eq_groups, &col_c_expr)); + + // This is a new set of equality. Hence equivalent class count should be 2. + eq_properties.add_equal_conditions(&col_x_expr, &col_y_expr); + assert_eq!(eq_properties.eq_group().len(), 2); + + // This equality bridges distinct equality sets. + // Hence equivalent class count should decrease from 2 to 1. + eq_properties.add_equal_conditions(&col_x_expr, &col_a_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; + assert_eq!(eq_groups.len(), 5); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + assert!(physical_exprs_contains(eq_groups, &col_c_expr)); + assert!(physical_exprs_contains(eq_groups, &col_x_expr)); + assert!(physical_exprs_contains(eq_groups, &col_y_expr)); Ok(()) } @@ -1283,11 +1422,8 @@ mod tests { Field::new("c", DataType::Int64, true), ])); - let mut input_properties = EquivalenceProperties::new(input_schema); - let new_condition = (&Column::new("a", 0), &Column::new("b", 1)); - input_properties.add_equal_conditions(new_condition); - let new_condition = (&Column::new("b", 1), &Column::new("c", 2)); - input_properties.add_equal_conditions(new_condition); + let input_properties = EquivalenceProperties::new(input_schema.clone()); + let col_a = col("a", &input_schema)?; let out_schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::Int64, true), @@ -1296,106 +1432,557 @@ mod tests { Field::new("a4", DataType::Int64, true), ])); - let mut alias_map = HashMap::new(); - alias_map.insert( - Column::new("a", 0), - vec![ - Column::new("a1", 0), - Column::new("a2", 1), - Column::new("a3", 2), - Column::new("a4", 3), - ], - ); - let mut out_properties = EquivalenceProperties::new(out_schema); + // a as a1, a as a2, a as a3, a as a3 + let col_a1 = &col("a1", &out_schema)?; + let col_a2 = &col("a2", &out_schema)?; + let col_a3 = &col("a3", &out_schema)?; + let col_a4 = &col("a4", &out_schema)?; + let projection_mapping = vec![ + (col_a.clone(), col_a1.clone()), + (col_a.clone(), col_a2.clone()), + (col_a.clone(), col_a3.clone()), + (col_a.clone(), col_a4.clone()), + ]; + let out_properties = input_properties.project(&projection_mapping, out_schema); - project_equivalence_properties(input_properties, &alias_map, &mut out_properties); - assert_eq!(out_properties.classes().len(), 1); - assert_eq!(out_properties.classes()[0].len(), 4); - assert!(out_properties.classes()[0].contains(&Column::new("a1", 0))); - assert!(out_properties.classes()[0].contains(&Column::new("a2", 1))); - assert!(out_properties.classes()[0].contains(&Column::new("a3", 2))); - assert!(out_properties.classes()[0].contains(&Column::new("a4", 3))); + // At the output a1=a2=a3=a4 + assert_eq!(out_properties.eq_group().len(), 1); + let eq_class = &out_properties.eq_group().classes[0]; + assert_eq!(eq_class.len(), 4); + assert!(physical_exprs_contains(eq_class, col_a1)); + assert!(physical_exprs_contains(eq_class, col_a2)); + assert!(physical_exprs_contains(eq_class, col_a3)); + assert!(physical_exprs_contains(eq_class, col_a4)); Ok(()) } #[test] - fn test_collapse_vec() -> Result<()> { - assert_eq!(collapse_vec(vec![1, 2, 3]), vec![1, 2, 3]); - assert_eq!(collapse_vec(vec![1, 2, 3, 2, 3]), vec![1, 2, 3]); - assert_eq!(collapse_vec(vec![3, 1, 2, 3, 2, 3]), vec![3, 1, 2]); + fn test_ordering_satisfy() -> Result<()> { + let crude = vec![PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }]; + let finer = vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + ]; + // finer ordering satisfies, crude ordering should return true + let empty_schema = &Arc::new(Schema::empty()); + let mut eq_properties_finer = EquivalenceProperties::new(empty_schema.clone()); + eq_properties_finer.oeq_class.push(finer.clone()); + assert!(eq_properties_finer.ordering_satisfy(&crude)); + + // Crude ordering doesn't satisfy finer ordering. should return false + let mut eq_properties_crude = EquivalenceProperties::new(empty_schema.clone()); + eq_properties_crude.oeq_class.push(crude.clone()); + assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) } #[test] - fn test_get_compatible_ranges() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let option1 = SortOptions { + fn test_ordering_satisfy_with_equivalence() -> Result<()> { + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_g = &col("g", &test_schema)?; + let option_asc = SortOptions { descending: false, nulls_first: false, }; - let test_data = vec![ + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, 625, 5)?; + + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function + let requirements = vec![ + // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it + (vec![(col_a, option_asc)], true), + (vec![(col_a, option_desc)], false), + // Test whether equivalence works as expected + (vec![(col_c, option_asc)], true), + (vec![(col_c, option_desc)], false), + // Test whether ordering equivalence works as expected + (vec![(col_d, option_asc)], true), + (vec![(col_d, option_asc), (col_b, option_asc)], true), + (vec![(col_d, option_desc), (col_b, option_asc)], false), ( - vec![(col_a, Some(option1)), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - vec![(0, 1)], + vec![ + (col_e, option_desc), + (col_f, option_asc), + (col_g, option_asc), + ], + true, ), + (vec![(col_e, option_desc), (col_f, option_asc)], true), + (vec![(col_e, option_asc), (col_f, option_asc)], false), + (vec![(col_e, option_desc), (col_b, option_asc)], false), + (vec![(col_e, option_asc), (col_b, option_asc)], false), ( - vec![(col_a, None), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - vec![(0, 1)], + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_d, option_asc), + (col_b, option_asc), + ], + true, ), ( vec![ - (col_a, None), - (col_b, Some(option1)), - (col_a, Some(option1)), + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + (col_f, option_asc), ], - vec![(col_a, Some(option1))], - vec![(0, 1), (2, 3)], + true, ), - ]; - for (searched, to_search, expected) in test_data { - let searched = convert_to_requirement(&searched); - let to_search = convert_to_requirement(&to_search); - let expected = expected - .into_iter() - .map(|(start, end)| Range { start, end }) - .collect::>(); - assert_eq!(get_compatible_ranges(&searched, &to_search), expected); - } - Ok(()) - } - - #[test] - fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { - let join_type = JoinType::Inner; - - let options = SortOptions::default(); - let right_oeq_class = OrderingEquivalentClass::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("x", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 1)), - options, - }, - ], - vec![vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("w", 3)), - options, - }, - ]], - ); - + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_d, option_desc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_f, option_asc), + ], + false, + ), + ( + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_b, option_asc), + ], + false, + ), + (vec![(col_d, option_asc), (col_e, option_desc)], true), + ( + vec![ + (col_d, option_asc), + (col_c, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_f, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_c, option_asc), + (col_b, option_asc), + ], + true, + ), + ( + vec![ + (col_d, option_asc), + (col_e, option_desc), + (col_b, option_asc), + (col_f, option_asc), + ], + true, + ), + ]; + + for (cols, expected) in requirements { + let err_msg = format!("Error in test case:{cols:?}"); + let required = cols + .into_iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: expr.clone(), + options, + }) + .collect::>(); + + // Check expected result with experimental result. + assert_eq!( + is_table_same_after_sort( + required.clone(), + table_data_with_properties.clone() + )?, + expected + ); + assert_eq!( + eq_properties.ordering_satisfy(&required), + expected, + "{err_msg}" + ); + } + Ok(()) + } + + #[test] + fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { + const N_RANDOM_SCHEMA: usize = 5; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { + descending: false, + nulls_first: false, + }; + + for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; + let col_exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + ]; + + for n_req in 0..=col_exprs.len() { + for exprs in col_exprs.iter().combinations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: SORT_OPTIONS, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + table_data_with_properties.clone(), + )?; + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}", + requirement, expected + ); + // Check whether ordering_satisfy API result and + // experimental result matches. + assert_eq!( + eq_properties.ordering_satisfy(&requirement), + expected, + "{}", + err_msg + ); + } + } + } + + Ok(()) + } + + #[test] + fn test_ordering_satisfy_different_lengths() -> Result<()> { + let test_schema = create_test_schema()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let options = SortOptions { + descending: false, + nulls_first: false, + }; + // a=c (e.g they are aliases). + let mut eq_properties = EquivalenceProperties::new(test_schema); + eq_properties.add_equal_conditions(col_a, col_c); + + let orderings = vec![ + vec![(col_a, options)], + vec![(col_e, options)], + vec![(col_d, options), (col_f, options)], + ]; + let orderings = convert_to_orderings(&orderings); + + // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. + eq_properties.add_new_orderings(orderings); + + // First entry in the tuple is required ordering, second entry is the expected flag + // that indicates whether this required ordering is satisfied. + // ([a ASC], true) indicate a ASC requirement is already satisfied by existing orderings. + let test_cases = vec![ + // [c ASC, a ASC, e ASC], expected represents this requirement is satisfied + ( + vec![(col_c, options), (col_a, options), (col_e, options)], + true, + ), + (vec![(col_c, options), (col_b, options)], false), + (vec![(col_c, options), (col_d, options)], true), + ( + vec![(col_d, options), (col_f, options), (col_b, options)], + false, + ), + (vec![(col_d, options), (col_f, options)], true), + ]; + + for (reqs, expected) in test_cases { + let err_msg = + format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); + let reqs = convert_to_sort_exprs(&reqs); + assert_eq!( + eq_properties.ordering_satisfy(&reqs), + expected, + "{}", + err_msg + ); + } + + Ok(()) + } + + #[test] + fn test_bridge_groups() -> Result<()> { + // First entry in the tuple is argument, second entry is the bridged result + let test_cases = vec![ + // ------- TEST CASE 1 -----------// + ( + vec![vec![1, 2, 3], vec![2, 4, 5], vec![11, 12, 9], vec![7, 6, 5]], + // Expected is compared with set equality. Order of the specific results may change. + vec![vec![1, 2, 3, 4, 5, 6, 7], vec![9, 11, 12]], + ), + // ------- TEST CASE 2 -----------// + ( + vec![vec![1, 2, 3], vec![3, 4, 5], vec![9, 8, 7], vec![7, 6, 5]], + // Expected + vec![vec![1, 2, 3, 4, 5, 6, 7, 8, 9]], + ), + ]; + for (entries, expected) in test_cases { + let entries = entries + .into_iter() + .map(|entry| entry.into_iter().map(lit).collect::>()) + .collect::>(); + let expected = expected + .into_iter() + .map(|entry| entry.into_iter().map(lit).collect::>()) + .collect::>(); + let mut eq_groups = EquivalenceGroup::new(entries.clone()); + eq_groups.bridge_classes(); + let eq_groups = eq_groups.classes; + let err_msg = format!( + "error in test entries: {:?}, expected: {:?}, actual:{:?}", + entries, expected, eq_groups + ); + assert_eq!(eq_groups.len(), expected.len(), "{}", err_msg); + for idx in 0..eq_groups.len() { + assert!( + physical_exprs_bag_equal(&eq_groups[idx], &expected[idx]), + "{}", + err_msg + ); + } + } + Ok(()) + } + + #[test] + fn test_remove_redundant_entries_eq_group() -> Result<()> { + let entries = vec![ + vec![lit(1), lit(1), lit(2)], + // This group is meaningless should be removed + vec![lit(3), lit(3)], + vec![lit(4), lit(5), lit(6)], + ]; + // Given equivalences classes are not in succinct form. + // Expected form is the most plain representation that is functionally same. + let expected = vec![vec![lit(1), lit(2)], vec![lit(4), lit(5), lit(6)]]; + let mut eq_groups = EquivalenceGroup::new(entries); + eq_groups.remove_redundant_entries(); + + let eq_groups = eq_groups.classes; + assert_eq!(eq_groups.len(), expected.len()); + assert_eq!(eq_groups.len(), 2); + + assert!(physical_exprs_equal(&eq_groups[0], &expected[0])); + assert!(physical_exprs_equal(&eq_groups[1], &expected[1])); + Ok(()) + } + + #[test] + fn test_remove_redundant_entries_oeq_class() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + + // First entry in the tuple is the given orderings for the table + // Second entry is the simplest version of the given orderings that is functionally equivalent. + let test_cases = vec![ + // ------- TEST CASE 1 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + ], + ), + // ------- TEST CASE 2 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + ), + // ------- TEST CASE 3 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b DESC] + vec![(col_a, option_asc), (col_b, option_desc)], + // [a ASC] + vec![(col_a, option_asc)], + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b DESC] + vec![(col_a, option_asc), (col_b, option_desc)], + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + ), + // ------- TEST CASE 4 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC] + vec![(col_a, option_asc)], + ], + // EXPECTED orderings that is succinct. + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + ), + ]; + for (orderings, expected) in test_cases { + let orderings = convert_to_orderings(&orderings); + let expected = convert_to_orderings(&expected); + let actual = OrderingEquivalenceClass::new(orderings.clone()); + let actual = actual.orderings; + let err_msg = format!( + "orderings: {:?}, expected: {:?}, actual :{:?}", + orderings, expected, actual + ); + assert_eq!(actual.len(), expected.len(), "{}", err_msg); + for elem in actual { + assert!(expected.contains(&elem), "{}", err_msg); + } + } + + Ok(()) + } + + #[test] + fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { + let join_type = JoinType::Inner; + // Join right child schema + let child_fields: Fields = ["x", "y", "z", "w"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect(); + let child_schema = Schema::new(child_fields); + let col_x = &col("x", &child_schema)?; + let col_y = &col("y", &child_schema)?; + let col_z = &col("z", &child_schema)?; + let col_w = &col("w", &child_schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // [x ASC, y ASC], [z ASC, w ASC] + let orderings = vec![ + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], + ]; + let orderings = convert_to_orderings(&orderings); + // Right child ordering equivalences + let mut right_oeq_class = OrderingEquivalenceClass::new(orderings); + let left_columns_len = 4; let fields: Fields = ["a", "b", "c", "d", "x", "y", "z", "w"] @@ -1403,45 +1990,578 @@ mod tests { .map(|name| Field::new(name, DataType::Int32, true)) .collect(); - let mut join_eq_properties = - EquivalenceProperties::new(Arc::new(Schema::new(fields))); - join_eq_properties - .add_equal_conditions((&Column::new("a", 0), &Column::new("x", 4))); - join_eq_properties - .add_equal_conditions((&Column::new("d", 3), &Column::new("w", 7))); - - let result = get_updated_right_ordering_equivalent_class( + // Join Schema + let schema = Schema::new(fields); + let col_a = &col("a", &schema)?; + let col_d = &col("d", &schema)?; + let col_x = &col("x", &schema)?; + let col_y = &col("y", &schema)?; + let col_z = &col("z", &schema)?; + let col_w = &col("w", &schema)?; + + let mut join_eq_properties = EquivalenceProperties::new(Arc::new(schema)); + // a=x and d=w + join_eq_properties.add_equal_conditions(col_a, col_x); + join_eq_properties.add_equal_conditions(col_d, col_w); + + updated_right_ordering_equivalence_class( + &mut right_oeq_class, &join_type, - &right_oeq_class, left_columns_len, - &join_eq_properties, ); + join_eq_properties.add_ordering_equivalence_class(right_oeq_class); + let result = join_eq_properties.oeq_class().clone(); - let expected = OrderingEquivalentClass::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 5)), - options, - }, - ], - vec![vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 6)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options, - }, - ]], - ); + // [x ASC, y ASC], [z ASC, w ASC] + let orderings = vec![ + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], + ]; + let orderings = convert_to_orderings(&orderings); + let expected = OrderingEquivalenceClass::new(orderings); + + assert_eq!(result, expected); + + Ok(()) + } + + /// Checks if the table (RecordBatch) remains unchanged when sorted according to the provided `required_ordering`. + /// + /// The function works by adding a unique column of ascending integers to the original table. This column ensures + /// that rows that are otherwise indistinguishable (e.g., if they have the same values in all other columns) can + /// still be differentiated. When sorting the extended table, the unique column acts as a tie-breaker to produce + /// deterministic sorting results. + /// + /// If the table remains the same after sorting with the added unique column, it indicates that the table was + /// already sorted according to `required_ordering` to begin with. + fn is_table_same_after_sort( + mut required_ordering: Vec, + batch: RecordBatch, + ) -> Result { + // Clone the original schema and columns + let original_schema = batch.schema(); + let mut columns = batch.columns().to_vec(); + + // Create a new unique column + let n_row = batch.num_rows() as u64; + let unique_col = Arc::new(UInt64Array::from_iter_values(0..n_row)) as ArrayRef; + columns.push(unique_col.clone()); + + // Create a new schema with the added unique column + let unique_col_name = "unique"; + let unique_field = Arc::new(Field::new(unique_col_name, DataType::UInt64, false)); + let fields: Vec<_> = original_schema + .fields() + .iter() + .cloned() + .chain(std::iter::once(unique_field)) + .collect(); + let schema = Arc::new(Schema::new(fields)); + + // Create a new batch with the added column + let new_batch = RecordBatch::try_new(schema.clone(), columns)?; + + // Add the unique column to the required ordering to ensure deterministic results + required_ordering.push(PhysicalSortExpr { + expr: Arc::new(Column::new(unique_col_name, original_schema.fields().len())), + options: Default::default(), + }); + + // Convert the required ordering to a list of SortColumn + let sort_columns: Vec<_> = required_ordering + .iter() + .filter_map(|order_expr| { + let col = order_expr.expr.as_any().downcast_ref::()?; + let col_index = schema.column_with_name(col.name())?.0; + Some(SortColumn { + values: new_batch.column(col_index).clone(), + options: Some(order_expr.options), + }) + }) + .collect(); + + // Check if the indices after sorting match the initial ordering + let sorted_indices = lexsort_to_indices(&sort_columns, None)?; + let original_indices = UInt32Array::from_iter_values(0..n_row as u32); + + Ok(sorted_indices == original_indices) + } + + // If we already generated a random result for one of the + // expressions in the equivalence classes. For other expressions in the same + // equivalence class use same result. This util gets already calculated result, when available. + fn get_representative_arr( + eq_group: &[Arc], + existing_vec: &[Option], + schema: SchemaRef, + ) -> Option { + for expr in eq_group.iter() { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + if let Some(res) = &existing_vec[idx] { + return Some(res.clone()); + } + } + None + } + + // Generate a table that satisfies the given equivalence properties; i.e. + // equivalences, ordering equivalences, and constants. + fn generate_table_for_eq_properties( + eq_properties: &EquivalenceProperties, + n_elem: usize, + n_distinct: usize, + ) -> Result { + let mut rng = StdRng::seed_from_u64(23); + + let schema = eq_properties.schema(); + let mut schema_vec = vec![None; schema.fields.len()]; + + // Utility closure to generate random array + let mut generate_random_array = |num_elems: usize, max_val: usize| -> ArrayRef { + let values: Vec = (0..num_elems) + .map(|_| rng.gen_range(0..max_val) as u64) + .collect(); + Arc::new(UInt64Array::from_iter_values(values)) + }; + + // Fill constant columns + for constant in &eq_properties.constants { + let col = constant.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = + Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; + schema_vec[idx] = Some(arr); + } + + // Fill columns based on ordering equivalences + for ordering in eq_properties.oeq_class.iter() { + let (sort_columns, indices): (Vec<_>, Vec<_>) = ordering + .iter() + .map(|PhysicalSortExpr { expr, options }| { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = generate_random_array(n_elem, n_distinct); + ( + SortColumn { + values: arr, + options: Some(*options), + }, + idx, + ) + }) + .unzip(); + + let sort_arrs = arrow::compute::lexsort(&sort_columns, None)?; + for (idx, arr) in izip!(indices, sort_arrs) { + schema_vec[idx] = Some(arr); + } + } + + // Fill columns based on equivalence groups + for eq_group in eq_properties.eq_group.iter() { + let representative_array = + get_representative_arr(eq_group, &schema_vec, schema.clone()) + .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); + + for expr in eq_group { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + schema_vec[idx] = Some(representative_array.clone()); + } + } - assert_eq!(result.head(), expected.head()); - assert_eq!(result.others(), expected.others()); + let res: Vec<_> = schema_vec + .into_iter() + .zip(schema.fields.iter()) + .map(|(elem, field)| { + ( + field.name(), + // Generate random values for columns that do not occur in any of the groups (equivalence, ordering equivalence, constants) + elem.unwrap_or_else(|| generate_random_array(n_elem, n_distinct)), + ) + }) + .collect(); + + Ok(RecordBatch::try_from_iter(res)?) + } + + #[test] + fn test_schema_normalize_expr_with_equivalence() -> Result<()> { + let col_a = &Column::new("a", 0); + let col_b = &Column::new("b", 1); + let col_c = &Column::new("c", 2); + // Assume that column a and c are aliases. + let (_test_schema, eq_properties) = create_test_params()?; + + let col_a_expr = Arc::new(col_a.clone()) as Arc; + let col_b_expr = Arc::new(col_b.clone()) as Arc; + let col_c_expr = Arc::new(col_c.clone()) as Arc; + // Test cases for equivalence normalization, + // First entry in the tuple is argument, second entry is expected result after normalization. + let expressions = vec![ + // Normalized version of the column a and c should go to a + // (by convention all the expressions inside equivalence class are mapped to the first entry + // in this case a is the first entry in the equivalence class.) + (&col_a_expr, &col_a_expr), + (&col_c_expr, &col_a_expr), + // Cannot normalize column b + (&col_b_expr, &col_b_expr), + ]; + let eq_group = eq_properties.eq_group(); + for (expr, expected_eq) in expressions { + assert!( + expected_eq.eq(&eq_group.normalize_expr(expr.clone())), + "error in test: expr: {expr:?}" + ); + } + + Ok(()) + } + + #[test] + fn test_schema_normalize_sort_requirement_with_equivalence() -> Result<()> { + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + // Assume that column a and c are aliases. + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + + // Test cases for equivalence normalization + // First entry in the tuple is PhysicalSortRequirement, second entry in the tuple is + // expected PhysicalSortRequirement after normalization. + let test_cases = vec![ + (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), + // In the normalized version column c should be replace with column a + (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), + (vec![(col_c, None)], vec![(col_a, None)]), + (vec![(col_d, Some(option1))], vec![(col_d, Some(option1))]), + ]; + for (reqs, expected) in test_cases.into_iter() { + let reqs = convert_to_sort_reqs(&reqs); + let expected = convert_to_sort_reqs(&expected); + + let normalized = eq_properties.normalize_sort_requirements(&reqs); + assert!( + expected.eq(&normalized), + "error in test: reqs: {reqs:?}, expected: {expected:?}, normalized: {normalized:?}" + ); + } + + Ok(()) + } + + #[test] + fn test_normalize_sort_reqs() -> Result<()> { + // Schema satisfies following properties + // a=c + // and following orderings are valid + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + let (test_schema, eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function + let requirements = vec![ + ( + vec![(col_a, Some(option_asc))], + vec![(col_a, Some(option_asc))], + ), + ( + vec![(col_a, Some(option_desc))], + vec![(col_a, Some(option_desc))], + ), + (vec![(col_a, None)], vec![(col_a, None)]), + // Test whether equivalence works as expected + ( + vec![(col_c, Some(option_asc))], + vec![(col_a, Some(option_asc))], + ), + (vec![(col_c, None)], vec![(col_a, None)]), + // Test whether ordering equivalence works as expected + ( + vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], + vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], + ), + ( + vec![(col_d, None), (col_b, None)], + vec![(col_d, None), (col_b, None)], + ), + ( + vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], + vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], + ), + // We should be able to normalize in compatible requirements also (not exactly equal) + ( + vec![(col_e, Some(option_desc)), (col_f, None)], + vec![(col_e, Some(option_desc)), (col_f, None)], + ), + ( + vec![(col_e, None), (col_f, None)], + vec![(col_e, None), (col_f, None)], + ), + ]; + + for (reqs, expected_normalized) in requirements.into_iter() { + let req = convert_to_sort_reqs(&reqs); + let expected_normalized = convert_to_sort_reqs(&expected_normalized); + + assert_eq!( + eq_properties.normalize_sort_requirements(&req), + expected_normalized + ); + } + + Ok(()) + } + + #[test] + fn test_get_finer() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let eq_properties = EquivalenceProperties::new(schema); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // First entry, and second entry are the physical sort requirement that are argument for get_finer_requirement. + // Third entry is the expected result. + let tests_cases = vec![ + // Get finer requirement between [a Some(ASC)] and [a None, b Some(ASC)] + // result should be [a Some(ASC), b Some(ASC)] + ( + vec![(col_a, Some(option_asc))], + vec![(col_a, None), (col_b, Some(option_asc))], + Some(vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))]), + ), + // Get finer requirement between [a Some(ASC), b Some(ASC), c Some(ASC)] and [a Some(ASC), b Some(ASC)] + // result should be [a Some(ASC), b Some(ASC), c Some(ASC)] + ( + vec![ + (col_a, Some(option_asc)), + (col_b, Some(option_asc)), + (col_c, Some(option_asc)), + ], + vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], + Some(vec![ + (col_a, Some(option_asc)), + (col_b, Some(option_asc)), + (col_c, Some(option_asc)), + ]), + ), + // Get finer requirement between [a Some(ASC), b Some(ASC)] and [a Some(ASC), b Some(DESC)] + // result should be None + ( + vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], + vec![(col_a, Some(option_asc)), (col_b, Some(option_desc))], + None, + ), + ]; + for (lhs, rhs, expected) in tests_cases { + let lhs = convert_to_sort_reqs(&lhs); + let rhs = convert_to_sort_reqs(&rhs); + let expected = expected.map(|expected| convert_to_sort_reqs(&expected)); + let finer = eq_properties.get_finer_requirement(&lhs, &rhs); + assert_eq!(finer, expected) + } + + Ok(()) + } + + #[test] + fn test_get_meet_ordering() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let eq_properties = EquivalenceProperties::new(schema); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let tests_cases = vec![ + // Get meet ordering between [a ASC] and [a ASC, b ASC] + // result should be [a ASC] + ( + vec![(col_a, option_asc)], + vec![(col_a, option_asc), (col_b, option_asc)], + Some(vec![(col_a, option_asc)]), + ), + // Get meet ordering between [a ASC] and [a DESC] + // result should be None. + (vec![(col_a, option_asc)], vec![(col_a, option_desc)], None), + // Get meet ordering between [a ASC, b ASC] and [a ASC, b DESC] + // result should be [a ASC]. + ( + vec![(col_a, option_asc), (col_b, option_asc)], + vec![(col_a, option_asc), (col_b, option_desc)], + Some(vec![(col_a, option_asc)]), + ), + ]; + for (lhs, rhs, expected) in tests_cases { + let lhs = convert_to_sort_exprs(&lhs); + let rhs = convert_to_sort_exprs(&rhs); + let expected = expected.map(|expected| convert_to_sort_exprs(&expected)); + let finer = eq_properties.get_meet_ordering(&lhs, &rhs); + assert_eq!(finer, expected) + } + + Ok(()) + } + + #[test] + fn test_find_longest_permutation() -> Result<()> { + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + // At below we add [d ASC, h DESC] also, for test purposes + let (test_schema, mut eq_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_h = &col("h", &test_schema)?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + // [d ASC, h ASC] also satisfies schema. + eq_properties.add_new_orderings([vec![ + PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }, + PhysicalSortExpr { + expr: col_h.clone(), + options: option_desc, + }, + ]]); + let test_cases = vec![ + // TEST CASE 1 + (vec![col_a], vec![(col_a, option_asc)]), + // TEST CASE 2 + (vec![col_c], vec![(col_c, option_asc)]), + // TEST CASE 3 + ( + vec![col_d, col_e, col_b], + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + ], + ), + // TEST CASE 4 + (vec![col_b], vec![]), + // TEST CASE 5 + (vec![col_d], vec![(col_d, option_asc)]), + ]; + for (exprs, expected) in test_cases { + let exprs = exprs.into_iter().cloned().collect::>(); + let expected = convert_to_sort_exprs(&expected); + let (actual, _) = eq_properties.find_longest_permutation(&exprs); + assert_eq!(actual, expected); + } + + Ok(()) + } + + #[test] + fn test_update_ordering() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + ]); + + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // b=a (e.g they are aliases) + eq_properties.add_equal_conditions(col_b, col_a); + // [b ASC], [d ASC] + eq_properties.add_new_orderings(vec![ + vec![PhysicalSortExpr { + expr: col_b.clone(), + options: option_asc, + }], + vec![PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }], + ]); + + let test_cases = vec![ + // d + b + ( + Arc::new(BinaryExpr::new( + col_d.clone(), + Operator::Plus, + col_b.clone(), + )) as Arc, + SortProperties::Ordered(option_asc), + ), + // b + (col_b.clone(), SortProperties::Ordered(option_asc)), + // a + (col_a.clone(), SortProperties::Ordered(option_asc)), + // a + c + ( + Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_c.clone(), + )), + SortProperties::Unordered, + ), + ]; + for (expr, expected) in test_cases { + let expr_ordering = ExprOrdering::new(expr.clone()); + let expr_ordering = expr_ordering + .transform_up(&|expr| update_ordering(expr, &eq_properties))?; + let err_msg = format!( + "expr:{:?}, expected: {:?}, actual: {:?}", + expr, expected, expr_ordering.state + ); + assert_eq!(expr_ordering.state, expected, "{}", err_msg); + } Ok(()) } diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 643bbfd820a6..8d55fb70bd9e 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -22,8 +22,7 @@ use std::fmt::Debug; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::physical_expr::down_cast_any_ref; -use crate::utils::expr_list_eq_any_order; +use crate::physical_expr::{down_cast_any_ref, physical_exprs_bag_equal}; use crate::PhysicalExpr; use arrow::array::*; @@ -410,7 +409,7 @@ impl PartialEq for InListExpr { .downcast_ref::() .map(|x| { self.expr.eq(&x.expr) - && expr_list_eq_any_order(&self.list, &x.list) + && physical_exprs_bag_equal(&self.list, &x.list) && self.negated == x.negated }) .unwrap_or(false) diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 977542bd8e66..fffa8f602d87 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -53,23 +53,16 @@ pub use aggregate::groups_accumulator::{ }; pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; -pub use equivalence::{ - add_offset_to_expr, add_offset_to_lex_ordering, - ordering_equivalence_properties_helper, project_equivalence_properties, - project_ordering_equivalence_properties, EquivalenceProperties, EquivalentClass, - OrderingEquivalenceProperties, OrderingEquivalentClass, -}; - +pub use equivalence::EquivalenceProperties; pub use partitioning::{Distribution, Partitioning}; -pub use physical_expr::{physical_exprs_contains, PhysicalExpr, PhysicalExprRef}; +pub use physical_expr::{ + physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, + PhysicalExpr, PhysicalExprRef, +}; pub use planner::create_physical_expr; pub use scalar_function::ScalarFunctionExpr; pub use sort_expr::{ - LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalSortExpr, + LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; -pub use sort_properties::update_ordering; -pub use utils::{ - expr_list_eq_any_order, expr_list_eq_strict_order, - normalize_out_expr_with_columns_map, reverse_order_bys, split_conjunction, -}; +pub use utils::{reverse_order_bys, split_conjunction}; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 773eac40dc8a..6a8fca4a1543 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -20,7 +20,7 @@ use std::fmt; use std::sync::Arc; -use crate::{expr_list_eq_strict_order, EquivalenceProperties, PhysicalExpr}; +use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// Partitioning schemes supported by operators. #[derive(Debug, Clone)] @@ -66,7 +66,7 @@ impl Partitioning { pub fn satisfy EquivalenceProperties>( &self, required: Distribution, - equal_properties: F, + eq_properties: F, ) -> bool { match required { Distribution::UnspecifiedDistribution => true, @@ -78,31 +78,28 @@ impl Partitioning { // then we need to have the partition count and hash functions validation. Partitioning::Hash(partition_exprs, _) => { let fast_match = - expr_list_eq_strict_order(&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 eq_properties + // and normalize both exprs based on the equivalent groups. if !fast_match { - let eq_properties = equal_properties(); - let eq_classes = eq_properties.classes(); - if !eq_classes.is_empty() { + let eq_properties = eq_properties(); + let eq_groups = eq_properties.eq_group(); + if !eq_groups.is_empty() { let normalized_required_exprs = required_exprs .iter() - .map(|e| eq_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); let normalized_partition_exprs = partition_exprs .iter() - .map(|e| eq_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); - expr_list_eq_strict_order( + return physical_exprs_equal( &normalized_required_exprs, &normalized_partition_exprs, - ) - } else { - fast_match + ); } - } else { - fast_match } + fast_match } _ => false, } @@ -120,7 +117,7 @@ impl PartialEq for Partitioning { Partitioning::RoundRobinBatch(count2), ) if count1 == count2 => true, (Partitioning::Hash(exprs1, count1), Partitioning::Hash(exprs2, count2)) - if expr_list_eq_strict_order(exprs1, exprs2) && (count1 == count2) => + if physical_exprs_equal(exprs1, exprs2) && (count1 == count2) => { true } @@ -158,15 +155,13 @@ impl Distribution { #[cfg(test)] mod tests { - use crate::expressions::Column; + use std::sync::Arc; use super::*; - use arrow::datatypes::DataType; - use arrow::datatypes::Field; - use arrow::datatypes::Schema; - use datafusion_common::Result; + use crate::expressions::Column; - use std::sync::Arc; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::Result; #[test] fn partitioning_satisfy_distribution() -> Result<()> { diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 11fa6c899621..79cbe6828b64 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -32,6 +32,8 @@ use datafusion_common::utils::DataPtr; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; +use itertools::izip; + /// Expression that can be evaluated against a RecordBatch /// A Physical expression knows its type, nullability and how to evaluate itself. pub trait PhysicalExpr: Send + Sync + Display + Debug + PartialEq { @@ -226,17 +228,79 @@ pub fn physical_exprs_contains( .any(|physical_expr| physical_expr.eq(expr)) } +/// Checks whether the given slices have any common entries. +pub fn have_common_entries( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + lhs.iter().any(|expr| physical_exprs_contains(rhs, expr)) +} + +/// Checks whether the given physical expression slices are equal. +pub fn physical_exprs_equal( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + lhs.len() == rhs.len() && izip!(lhs, rhs).all(|(lhs, rhs)| lhs.eq(rhs)) +} + +/// Checks whether the given physical expression slices are equal in the sense +/// of bags (multi-sets), disregarding their orderings. +pub fn physical_exprs_bag_equal( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + // TODO: Once we can use `HashMap`s with `Arc`, this + // function should use a `HashMap` to reduce computational complexity. + if lhs.len() == rhs.len() { + let mut rhs_vec = rhs.to_vec(); + for expr in lhs { + if let Some(idx) = rhs_vec.iter().position(|e| expr.eq(e)) { + rhs_vec.swap_remove(idx); + } else { + return false; + } + } + true + } else { + false + } +} + +/// This utility function removes duplicates from the given `exprs` vector. +/// Note that this function does not necessarily preserve its input ordering. +pub fn deduplicate_physical_exprs(exprs: &mut Vec>) { + // TODO: Once we can use `HashSet`s with `Arc`, this + // function should use a `HashSet` to reduce computational complexity. + // See issue: https://github.com/apache/arrow-datafusion/issues/8027 + let mut idx = 0; + while idx < exprs.len() { + let mut rest_idx = idx + 1; + while rest_idx < exprs.len() { + if exprs[idx].eq(&exprs[rest_idx]) { + exprs.swap_remove(rest_idx); + } else { + rest_idx += 1; + } + } + idx += 1; + } +} + #[cfg(test)] mod tests { use std::sync::Arc; use crate::expressions::{Column, Literal}; - use crate::physical_expr::{physical_exprs_contains, PhysicalExpr}; + use crate::physical_expr::{ + deduplicate_physical_exprs, have_common_entries, physical_exprs_bag_equal, + physical_exprs_contains, physical_exprs_equal, PhysicalExpr, + }; - use datafusion_common::{Result, ScalarValue}; + use datafusion_common::ScalarValue; #[test] - fn test_physical_exprs_contains() -> Result<()> { + fn test_physical_exprs_contains() { let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) as Arc; let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) @@ -268,6 +332,142 @@ mod tests { // below expressions are not inside physical_exprs assert!(!physical_exprs_contains(&physical_exprs, &col_c_expr)); assert!(!physical_exprs_contains(&physical_exprs, &lit1)); - Ok(()) + } + + #[test] + fn test_have_common_entries() { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + + let vec1 = vec![lit_true.clone(), lit_false.clone()]; + let vec2 = vec![lit_true.clone(), col_b_expr.clone()]; + let vec3 = vec![lit2.clone(), lit1.clone()]; + + // lit_true is common + assert!(have_common_entries(&vec1, &vec2)); + // there is no common entry + assert!(!have_common_entries(&vec1, &vec3)); + assert!(!have_common_entries(&vec2, &vec3)); + } + + #[test] + fn test_physical_exprs_equal() { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + + let vec1 = vec![lit_true.clone(), lit_false.clone()]; + let vec2 = vec![lit_true.clone(), col_b_expr.clone()]; + let vec3 = vec![lit2.clone(), lit1.clone()]; + let vec4 = vec![lit_true.clone(), lit_false.clone()]; + + // these vectors are same + assert!(physical_exprs_equal(&vec1, &vec1)); + assert!(physical_exprs_equal(&vec1, &vec4)); + assert!(physical_exprs_bag_equal(&vec1, &vec1)); + assert!(physical_exprs_bag_equal(&vec1, &vec4)); + + // these vectors are different + assert!(!physical_exprs_equal(&vec1, &vec2)); + assert!(!physical_exprs_equal(&vec1, &vec3)); + assert!(!physical_exprs_bag_equal(&vec1, &vec2)); + assert!(!physical_exprs_bag_equal(&vec1, &vec3)); + } + + #[test] + fn test_physical_exprs_set_equal() { + let list1: Vec> = vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]; + let list2: Vec> = vec![ + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("a", 0)), + ]; + assert!(!physical_exprs_bag_equal( + list1.as_slice(), + list2.as_slice() + )); + assert!(!physical_exprs_bag_equal( + list2.as_slice(), + list1.as_slice() + )); + assert!(!physical_exprs_equal(list1.as_slice(), list2.as_slice())); + assert!(!physical_exprs_equal(list2.as_slice(), list1.as_slice())); + + let list3: Vec> = vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("c", 2)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]; + let list4: Vec> = vec![ + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("c", 2)), + Arc::new(Column::new("a", 0)), + ]; + assert!(physical_exprs_bag_equal(list3.as_slice(), list4.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list3.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list4.as_slice())); + assert!(!physical_exprs_equal(list3.as_slice(), list4.as_slice())); + assert!(!physical_exprs_equal(list4.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list3.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list4.as_slice())); + } + + #[test] + fn test_deduplicate_physical_exprs() { + let lit_true = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc); + let lit_false = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc); + let lit4 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) + as Arc); + let lit2 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) + as Arc); + let col_a_expr = &(Arc::new(Column::new("a", 0)) as Arc); + let col_b_expr = &(Arc::new(Column::new("b", 1)) as Arc); + + // First vector in the tuple is arguments, second one is the expected value. + let test_cases = vec![ + // ---------- TEST CASE 1----------// + ( + vec![ + lit_true, lit_false, lit4, lit2, col_a_expr, col_a_expr, col_b_expr, + lit_true, lit2, + ], + vec![lit_true, lit_false, lit4, lit2, col_a_expr, col_b_expr], + ), + // ---------- TEST CASE 2----------// + ( + vec![lit_true, lit_true, lit_false, lit4], + vec![lit_true, lit4, lit_false], + ), + ]; + for (exprs, expected) in test_cases { + let mut exprs = exprs.into_iter().cloned().collect::>(); + let expected = expected.into_iter().cloned().collect::>(); + deduplicate_physical_exprs(&mut exprs); + assert!(physical_exprs_equal(&exprs, &expected)); + } } } diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 23f516c81851..6fc42bc94172 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -30,15 +30,13 @@ //! to a function that supports f64, it is coerced to f64. use std::any::Any; -use std::fmt::Debug; -use std::fmt::{self, Formatter}; +use std::fmt::{self, Debug, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::Arc; use crate::functions::out_ordering; -use crate::physical_expr::down_cast_any_ref; +use crate::physical_expr::{down_cast_any_ref, physical_exprs_equal}; use crate::sort_properties::SortProperties; -use crate::utils::expr_list_eq_strict_order; use crate::PhysicalExpr; use arrow::datatypes::{DataType, Schema}; @@ -195,7 +193,7 @@ impl PartialEq for ScalarFunctionExpr { .downcast_ref::() .map(|x| { self.name == x.name - && expr_list_eq_strict_order(&self.args, &x.args) + && physical_exprs_equal(&self.args, &x.args) && self.return_type == x.return_type }) .unwrap_or(false) diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 3b3221289d31..664a6b65b7f7 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -25,8 +25,8 @@ use crate::PhysicalExpr; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::record_batch::RecordBatch; -use datafusion_common::exec_err; -use datafusion_common::{DataFusionError, Result}; +use arrow_schema::Schema; +use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; /// Represents Sort operation for a column in a RecordBatch @@ -77,18 +77,26 @@ impl PhysicalSortExpr { }) } - /// Check whether sort expression satisfies [`PhysicalSortRequirement`]. - /// - /// If sort options is Some in `PhysicalSortRequirement`, `expr` - /// and `options` field are compared for equality. - /// - /// If sort options is None in `PhysicalSortRequirement`, only - /// `expr` is compared for equality. - pub fn satisfy(&self, requirement: &PhysicalSortRequirement) -> bool { + /// Checks whether this sort expression satisfies the given `requirement`. + /// If sort options are unspecified in `requirement`, only expressions are + /// compared for inequality. + pub fn satisfy( + &self, + requirement: &PhysicalSortRequirement, + schema: &Schema, + ) -> bool { + // If the column is not nullable, NULLS FIRST/LAST is not important. + let nullable = self.expr.nullable(schema).unwrap_or(true); self.expr.eq(&requirement.expr) - && requirement - .options - .map_or(true, |opts| self.options == opts) + && if nullable { + requirement + .options + .map_or(true, |opts| self.options == opts) + } else { + requirement + .options + .map_or(true, |opts| self.options.descending == opts.descending) + } } /// Returns a [`Display`]able list of `PhysicalSortExpr`. @@ -248,11 +256,18 @@ fn to_str(options: &SortOptions) -> &str { } } -///`LexOrdering` is a type alias for lexicographical ordering definition`Vec` +///`LexOrdering` is an alias for the type `Vec`, which represents +/// a lexicographical ordering. pub type LexOrdering = Vec; -///`LexOrderingRef` is a type alias for lexicographical ordering reference &`[PhysicalSortExpr]` +///`LexOrderingRef` is an alias for the type &`[PhysicalSortExpr]`, which represents +/// a reference to a lexicographical ordering. pub type LexOrderingRef<'a> = &'a [PhysicalSortExpr]; -///`LexOrderingReq` is a type alias for lexicographical ordering requirement definition`Vec` -pub type LexOrderingReq = Vec; +///`LexRequirement` is an alias for the type `Vec`, which +/// represents a lexicographical ordering requirement. +pub type LexRequirement = Vec; + +///`LexRequirementRef` is an alias for the type &`[PhysicalSortRequirement]`, which +/// represents a reference to a lexicographical ordering requirement. +pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 8ae3379218fb..a3b201f84e9d 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -17,14 +17,10 @@ use std::{ops::Neg, sync::Arc}; -use crate::expressions::Column; -use crate::utils::get_indices_of_matching_sort_exprs_with_order_eq; -use crate::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, -}; +use crate::PhysicalExpr; use arrow_schema::SortOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; use itertools::Itertools; @@ -224,57 +220,3 @@ impl TreeNode for ExprOrdering { } } } - -/// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. -/// The node is either a leaf node, or an intermediate node: -/// - If it is a leaf node, the children states are `None`. We directly find -/// the order of the node by looking at the given sort expression and equivalence -/// properties if it is a `Column` leaf, or we mark it as unordered. In the case -/// of a `Literal` leaf, we mark it as singleton so that it can cooperate with -/// some ordered columns at the upper steps. -/// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` -/// and operator has its own rules about how to propagate the children orderings. -/// However, before the children order propagation, it is checked that whether -/// the intermediate node can be directly matched with the sort expression. If there -/// is a match, the sort expression emerges at that node immediately, discarding -/// the order coming from the children. -pub fn update_ordering( - mut node: ExprOrdering, - sort_expr: &PhysicalSortExpr, - equal_properties: &EquivalenceProperties, - ordering_equal_properties: &OrderingEquivalenceProperties, -) -> Result> { - // If we can directly match a sort expr with the current node, we can set - // its state and return early. - // TODO: If there is a PhysicalExpr other than a Column at this node (e.g. - // a BinaryExpr like a + b), and there is an ordering equivalence of - // it (let's say like c + d), we actually can find it at this step. - if sort_expr.expr.eq(&node.expr) { - node.state = SortProperties::Ordered(sort_expr.options); - return Ok(Transformed::Yes(node)); - } - - if !node.expr.children().is_empty() { - // We have an intermediate (non-leaf) node, account for its children: - node.state = node.expr.get_ordering(&node.children_states); - } else if let Some(column) = node.expr.as_any().downcast_ref::() { - // We have a Column, which is one of the two possible leaf node types: - node.state = get_indices_of_matching_sort_exprs_with_order_eq( - &[sort_expr.clone()], - &[column.clone()], - equal_properties, - ordering_equal_properties, - ) - .map(|(sort_options, _)| { - SortProperties::Ordered(SortOptions { - descending: sort_options[0].descending, - nulls_first: sort_options[0].nulls_first, - }) - }) - .unwrap_or(SortProperties::Unordered); - } else { - // We have a Literal, which is the other possible leaf node type: - node.state = node.expr.get_ordering(&[]); - } - Ok(Transformed::Yes(node)) -} diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index b38117d206cc..a341f5d9bc2f 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -15,62 +15,25 @@ // specific language governing permissions and limitations // under the License. -use crate::equivalence::{EquivalenceProperties, OrderingEquivalenceProperties}; -use crate::expressions::{BinaryExpr, Column, UnKnownColumn}; -use crate::sort_properties::{ExprOrdering, SortProperties}; -use crate::update_ordering; -use crate::{PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement}; +use std::borrow::Borrow; +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +use crate::expressions::{BinaryExpr, Column}; +use crate::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use arrow::datatypes::SchemaRef; -use arrow_schema::SortOptions; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeRewriter, VisitRecursion, }; -use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::Result; use datafusion_expr::Operator; use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; -use std::borrow::Borrow; -use std::collections::HashMap; -use std::collections::HashSet; -use std::sync::Arc; - -/// Compare the two expr lists are equal no matter the order. -/// For example two InListExpr can be considered to be equals no matter the order: -/// -/// In('a','b','c') == In('c','b','a') -pub fn expr_list_eq_any_order( - list1: &[Arc], - list2: &[Arc], -) -> bool { - if list1.len() == list2.len() { - let mut expr_vec1 = list1.to_vec(); - let mut expr_vec2 = list2.to_vec(); - while let Some(expr1) = expr_vec1.pop() { - if let Some(idx) = expr_vec2.iter().position(|expr2| expr1.eq(expr2)) { - expr_vec2.swap_remove(idx); - } else { - break; - } - } - expr_vec1.is_empty() && expr_vec2.is_empty() - } else { - false - } -} - -/// Strictly compare the two expr lists are equal in the given order. -pub fn expr_list_eq_strict_order( - list1: &[Arc], - list2: &[Arc], -) -> bool { - list1.len() == list2.len() && list1.iter().zip(list2.iter()).all(|(e1, e2)| e1.eq(e2)) -} /// Assume the predicate is in the form of CNF, split the predicate to a Vec of PhysicalExprs. /// @@ -103,219 +66,6 @@ fn split_conjunction_impl<'a>( } } -/// Normalize the output expressions based on Columns Map. -/// -/// If there is a mapping in Columns Map, replace the Column in the output expressions with the 1st Column in the Columns Map. -/// Otherwise, replace the Column with a place holder of [UnKnownColumn] -/// -pub fn normalize_out_expr_with_columns_map( - expr: Arc, - columns_map: &HashMap>, -) -> Arc { - expr.clone() - .transform(&|expr| { - let normalized_form = match expr.as_any().downcast_ref::() { - Some(column) => columns_map - .get(column) - .map(|c| Arc::new(c[0].clone()) as _) - .or_else(|| Some(Arc::new(UnKnownColumn::new(column.name())) as _)), - None => None, - }; - Ok(if let Some(normalized_form) = normalized_form { - Transformed::Yes(normalized_form) - } else { - Transformed::No(expr) - }) - }) - .unwrap_or(expr) -} - -/// Transform `sort_exprs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` -/// Assume `eq_properties` states that `Column a` and `Column b` are aliases. -/// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are -/// ordering equivalent (in the sense that both describe the ordering of the table). -/// If the `sort_exprs` input to this function were `vec![b ASC, c ASC]`, -/// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_properties` -/// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `ordering_eq_properties`. -/// Standardized version `vec![d ASC]` is used in subsequent operations. -fn normalize_sort_exprs( - sort_exprs: &[PhysicalSortExpr], - eq_properties: &EquivalenceProperties, - ordering_eq_properties: &OrderingEquivalenceProperties, -) -> Vec { - let sort_requirements = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_exprs = normalize_sort_requirements( - &sort_requirements, - eq_properties, - ordering_eq_properties, - ); - PhysicalSortRequirement::to_sort_exprs(normalized_exprs) -} - -/// Transform `sort_reqs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` -/// Assume `eq_properties` states that `Column a` and `Column b` are aliases. -/// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are -/// ordering equivalent (in the sense that both describe the ordering of the table). -/// If the `sort_reqs` input to this function were `vec![b Some(ASC), c None]`, -/// This function converts `sort_exprs` `vec![b Some(ASC), c None]` to first `vec![a Some(ASC), c None]` after considering `eq_properties` -/// Then converts `vec![a Some(ASC), c None]` to `vec![d Some(ASC)]` after considering `ordering_eq_properties`. -/// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. -fn normalize_sort_requirements( - sort_reqs: &[PhysicalSortRequirement], - eq_properties: &EquivalenceProperties, - ordering_eq_properties: &OrderingEquivalenceProperties, -) -> Vec { - let normalized_sort_reqs = eq_properties.normalize_sort_requirements(sort_reqs); - ordering_eq_properties.normalize_sort_requirements(&normalized_sort_reqs) -} - -/// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. -pub fn ordering_satisfy< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: Option<&[PhysicalSortExpr]>, - required: Option<&[PhysicalSortExpr]>, - equal_properties: F, - ordering_equal_properties: F2, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => ordering_satisfy_concrete( - provided, - required, - equal_properties, - ordering_equal_properties, - ), - } -} - -/// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_concrete< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: &[PhysicalSortExpr], - required: &[PhysicalSortExpr], - equal_properties: F, - ordering_equal_properties: F2, -) -> bool { - let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - let required_normalized = - normalize_sort_exprs(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_exprs(provided, &eq_properties, &oeq_properties); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given == req) -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_requirement< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: Option<&[PhysicalSortExpr]>, - required: Option<&[PhysicalSortRequirement]>, - equal_properties: F, - ordering_equal_properties: F2, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => ordering_satisfy_requirement_concrete( - provided, - required, - equal_properties, - ordering_equal_properties, - ), - } -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_requirement_concrete< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: &[PhysicalSortExpr], - required: &[PhysicalSortRequirement], - equal_properties: F, - ordering_equal_properties: F2, -) -> bool { - let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - let required_normalized = - normalize_sort_requirements(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_exprs(provided, &eq_properties, &oeq_properties); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.satisfy(&req)) -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are equal or more -/// specific than the provided [`PhysicalSortRequirement`]s. -pub fn requirements_compatible< - F: FnOnce() -> OrderingEquivalenceProperties, - F2: FnOnce() -> EquivalenceProperties, ->( - provided: Option<&[PhysicalSortRequirement]>, - required: Option<&[PhysicalSortRequirement]>, - ordering_equal_properties: F, - equal_properties: F2, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => requirements_compatible_concrete( - provided, - required, - ordering_equal_properties, - equal_properties, - ), - } -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are equal or more -/// specific than the provided [`PhysicalSortRequirement`]s. -fn requirements_compatible_concrete< - F: FnOnce() -> OrderingEquivalenceProperties, - F2: FnOnce() -> EquivalenceProperties, ->( - provided: &[PhysicalSortRequirement], - required: &[PhysicalSortRequirement], - ordering_equal_properties: F, - equal_properties: F2, -) -> bool { - let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - - let required_normalized = - normalize_sort_requirements(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_requirements(provided, &eq_properties, &oeq_properties); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.compatible(&req)) -} - /// This function maps back requirement after ProjectionExec /// to the Executor for its input. // Specifically, `ProjectionExec` changes index of `Column`s in the schema of its input executor. @@ -358,19 +108,6 @@ pub fn convert_to_expr>( .collect() } -/// This function finds the indices of `targets` within `items`, taking into -/// account equivalences according to `equal_properties`. -pub fn get_indices_of_matching_exprs EquivalenceProperties>( - targets: &[Arc], - items: &[Arc], - equal_properties: F, -) -> Vec { - let eq_properties = equal_properties(); - let normalized_items = eq_properties.normalize_exprs(items); - let normalized_targets = eq_properties.normalize_exprs(targets); - get_indices_of_exprs_strict(normalized_targets, &normalized_items) -} - /// This function finds the indices of `targets` within `items` using strict /// equality. pub fn get_indices_of_exprs_strict>>( @@ -567,31 +304,6 @@ pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec EquivalenceProperties, - F2: Fn() -> OrderingEquivalenceProperties, ->( - req1: &'a [PhysicalSortExpr], - req2: &'a [PhysicalSortExpr], - eq_properties: F, - ordering_eq_properties: F2, -) -> Option<&'a [PhysicalSortExpr]> { - if ordering_satisfy_concrete(req1, req2, &eq_properties, &ordering_eq_properties) { - // Finer requirement is `provided`, since it satisfies the other: - return Some(req1); - } - if ordering_satisfy_concrete(req2, req1, &eq_properties, &ordering_eq_properties) { - // Finer requirement is `req`, since it satisfies the other: - return Some(req2); - } - // Neither `provided` nor `req` satisfies one another, they are incompatible. - None -} - /// Scatter `truthy` array by boolean mask. When the mask evaluates `true`, next values of `truthy` /// are taken, when the mask evaluates `false` values null values are filled. /// @@ -635,159 +347,6 @@ pub fn scatter(mask: &BooleanArray, truthy: &dyn Array) -> Result { Ok(make_array(data)) } -/// Return indices of each item in `required_exprs` inside `provided_exprs`. -/// All the items should be found inside `provided_exprs`. Found indices will -/// be a permutation of the range 0, 1, ..., N. For example, \[2,1,0\] is valid -/// (\[0,1,2\] is consecutive), but \[3,1,0\] is not valid (\[0,1,3\] is not -/// consecutive). -fn get_lexicographical_match_indices( - required_exprs: &[Arc], - provided_exprs: &[Arc], -) -> Option> { - let indices_of_equality = get_indices_of_exprs_strict(required_exprs, provided_exprs); - let mut ordered_indices = indices_of_equality.clone(); - ordered_indices.sort(); - let n_match = indices_of_equality.len(); - let first_n = longest_consecutive_prefix(ordered_indices); - (n_match == required_exprs.len() && first_n == n_match && n_match > 0) - .then_some(indices_of_equality) -} - -/// Attempts to find a full match between the required columns to be ordered (lexicographically), and -/// the provided sort options (lexicographically), while considering equivalence properties. -/// -/// It starts by normalizing members of both the required columns and the provided sort options. -/// If a full match is found, returns the sort options and indices of the matches. If no full match is found, -/// the function proceeds to check against ordering equivalence properties. If still no full match is found, -/// the function returns `None`. -pub fn get_indices_of_matching_sort_exprs_with_order_eq( - provided_sorts: &[PhysicalSortExpr], - required_columns: &[Column], - eq_properties: &EquivalenceProperties, - order_eq_properties: &OrderingEquivalenceProperties, -) -> Option<(Vec, Vec)> { - // Create a vector of `PhysicalSortRequirement`s from the required columns: - let sort_requirement_on_requirements = required_columns - .iter() - .map(|required_column| PhysicalSortRequirement { - expr: Arc::new(required_column.clone()) as _, - options: None, - }) - .collect::>(); - - let normalized_required = normalize_sort_requirements( - &sort_requirement_on_requirements, - eq_properties, - &OrderingEquivalenceProperties::new(order_eq_properties.schema()), - ); - let normalized_provided = normalize_sort_requirements( - &PhysicalSortRequirement::from_sort_exprs(provided_sorts.iter()), - eq_properties, - &OrderingEquivalenceProperties::new(order_eq_properties.schema()), - ); - - let provided_sorts = normalized_provided - .iter() - .map(|req| req.expr.clone()) - .collect::>(); - - let normalized_required_expr = normalized_required - .iter() - .map(|req| req.expr.clone()) - .collect::>(); - - if let Some(indices_of_equality) = - get_lexicographical_match_indices(&normalized_required_expr, &provided_sorts) - { - return Some(( - indices_of_equality - .iter() - .filter_map(|index| normalized_provided[*index].options) - .collect(), - indices_of_equality, - )); - } - - // We did not find all the expressions, consult ordering equivalence properties: - if let Some(oeq_class) = order_eq_properties.oeq_class() { - let head = oeq_class.head(); - for ordering in oeq_class.others().iter().chain(std::iter::once(head)) { - let order_eq_class_exprs = convert_to_expr(ordering); - if let Some(indices_of_equality) = get_lexicographical_match_indices( - &normalized_required_expr, - &order_eq_class_exprs, - ) { - return Some(( - indices_of_equality - .iter() - .map(|index| ordering[*index].options) - .collect(), - indices_of_equality, - )); - } - } - } - // If no match found, return `None`: - None -} - -/// Calculates the output orderings for a set of expressions within the context of a given -/// execution plan. The resulting orderings are all in the type of [`Column`], since these -/// expressions become [`Column`] after the projection step. The expressions having an alias -/// are renamed with those aliases in the returned [`PhysicalSortExpr`]'s. If an expression -/// is found to be unordered, the corresponding entry in the output vector is `None`. -/// -/// # Arguments -/// -/// * `expr` - A slice of tuples containing expressions and their corresponding aliases. -/// -/// * `input_output_ordering` - Output ordering of the input plan. -/// -/// * `input_equal_properties` - Equivalence properties of the columns in the input plan. -/// -/// * `input_ordering_equal_properties` - Ordering equivalence properties of the columns in the input plan. -/// -/// # Returns -/// -/// A `Result` containing a vector of optional [`PhysicalSortExpr`]'s. Each element of the -/// vector corresponds to an expression from the input slice. If an expression can be ordered, -/// the corresponding entry is `Some(PhysicalSortExpr)`. If an expression cannot be ordered, -/// the entry is `None`. -pub fn find_orderings_of_exprs( - expr: &[(Arc, String)], - input_output_ordering: Option<&[PhysicalSortExpr]>, - input_equal_properties: EquivalenceProperties, - input_ordering_equal_properties: OrderingEquivalenceProperties, -) -> Result>> { - let mut orderings: Vec> = vec![]; - if let Some(leading_ordering) = - input_output_ordering.and_then(|output_ordering| output_ordering.first()) - { - for (index, (expression, name)) in expr.iter().enumerate() { - let initial_expr = ExprOrdering::new(expression.clone()); - let transformed = initial_expr.transform_up(&|expr| { - update_ordering( - expr, - leading_ordering, - &input_equal_properties, - &input_ordering_equal_properties, - ) - })?; - if let SortProperties::Ordered(sort_options) = transformed.state { - orderings.push(Some(PhysicalSortExpr { - expr: Arc::new(Column::new(name, index)), - options: sort_options, - })); - } else { - orderings.push(None); - } - } - } else { - orderings.extend(expr.iter().map(|_| None)); - } - Ok(orderings) -} - /// Merge left and right sort expressions, checking for duplicates. pub fn merge_vectors( left: &[PhysicalSortExpr], @@ -807,9 +366,9 @@ mod tests { use std::sync::Arc; use super::*; - use crate::equivalence::OrderingEquivalenceProperties; + use crate::equivalence::EquivalenceProperties; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; - use crate::{OrderingEquivalentClass, PhysicalSortExpr}; + use crate::PhysicalSortExpr; use arrow::compute::SortOptions; use arrow_array::Int32Array; @@ -858,86 +417,6 @@ mod tests { } } - // Generate a schema which consists of 5 columns (a, b, c, d, e) - fn create_test_schema() -> Result { - let a = Field::new("a", DataType::Int32, true); - let b = Field::new("b", DataType::Int32, true); - let c = Field::new("c", DataType::Int32, true); - let d = Field::new("d", DataType::Int32, true); - let e = Field::new("e", DataType::Int32, true); - let f = Field::new("f", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); - - Ok(schema) - } - - fn create_test_params() -> Result<( - SchemaRef, - EquivalenceProperties, - OrderingEquivalenceProperties, - )> { - // Assume schema satisfies ordering a ASC NULLS LAST - // and d ASC NULLS LAST, b ASC NULLS LAST and e DESC NULLS FIRST, f ASC NULLS LAST, g ASC NULLS LAST - // Assume that column a and c are aliases. - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let col_g = &Column::new("g", 6); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - let test_schema = create_test_schema()?; - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - eq_properties.add_equal_conditions((col_a, col_c)); - let mut ordering_eq_properties = - OrderingEquivalenceProperties::new(test_schema.clone()); - ordering_eq_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], - &vec![ - PhysicalSortExpr { - expr: Arc::new(col_d.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_b.clone()), - options: option1, - }, - ], - )); - ordering_eq_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], - &vec![ - PhysicalSortExpr { - expr: Arc::new(col_e.clone()), - options: option2, - }, - PhysicalSortExpr { - expr: Arc::new(col_f.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_g.clone()), - options: option1, - }, - ], - )); - Ok((test_schema, eq_properties, ordering_eq_properties)) - } - #[test] fn test_build_dag() -> Result<()> { let schema = Schema::new(vec![ @@ -1016,9 +495,7 @@ mod tests { } #[test] - fn test_get_indices_of_matching_exprs() { - let empty_schema = &Arc::new(Schema::empty()); - let equal_properties = || EquivalenceProperties::new(empty_schema.clone()); + fn test_get_indices_of_exprs_strict() { let list1: Vec> = vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -1030,313 +507,8 @@ mod tests { Arc::new(Column::new("c", 2)), Arc::new(Column::new("a", 0)), ]; - assert_eq!( - get_indices_of_matching_exprs(&list1, &list2, equal_properties), - vec![2, 0, 1] - ); - assert_eq!( - get_indices_of_matching_exprs(&list2, &list1, equal_properties), - vec![1, 2, 0] - ); - } - - #[test] - fn expr_list_eq_test() -> Result<()> { - let list1: Vec> = vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]; - let list2: Vec> = vec![ - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("a", 0)), - ]; - assert!(!expr_list_eq_any_order(list1.as_slice(), list2.as_slice())); - assert!(!expr_list_eq_any_order(list2.as_slice(), list1.as_slice())); - - assert!(!expr_list_eq_strict_order( - list1.as_slice(), - list2.as_slice() - )); - assert!(!expr_list_eq_strict_order( - list2.as_slice(), - list1.as_slice() - )); - - let list3: Vec> = vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("c", 2)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]; - let list4: Vec> = vec![ - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("c", 2)), - Arc::new(Column::new("a", 0)), - ]; - assert!(expr_list_eq_any_order(list3.as_slice(), list4.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list3.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list4.as_slice())); - - assert!(!expr_list_eq_strict_order( - list3.as_slice(), - list4.as_slice() - )); - assert!(!expr_list_eq_strict_order( - list4.as_slice(), - list3.as_slice() - )); - assert!(expr_list_eq_any_order(list3.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list4.as_slice())); - - Ok(()) - } - - #[test] - fn test_ordering_satisfy() -> Result<()> { - let crude = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }]; - let crude = Some(&crude[..]); - let finer = vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - ]; - let finer = Some(&finer[..]); - let empty_schema = &Arc::new(Schema::empty()); - assert!(ordering_satisfy( - finer, - crude, - || { EquivalenceProperties::new(empty_schema.clone()) }, - || { OrderingEquivalenceProperties::new(empty_schema.clone()) }, - )); - assert!(!ordering_satisfy( - crude, - finer, - || { EquivalenceProperties::new(empty_schema.clone()) }, - || { OrderingEquivalenceProperties::new(empty_schema.clone()) }, - )); - Ok(()) - } - - #[test] - fn test_ordering_satisfy_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let col_g = &Column::new("g", 6); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - // The schema is ordered by a ASC NULLS LAST, b ASC NULLS LAST - let provided = vec![ - PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_b.clone()), - options: option1, - }, - ]; - let provided = Some(&provided[..]); - let (_test_schema, eq_properties, ordering_eq_properties) = create_test_params()?; - // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function - let requirements = vec![ - // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it - (vec![(col_a, option1)], true), - (vec![(col_a, option2)], false), - // Test whether equivalence works as expected - (vec![(col_c, option1)], true), - (vec![(col_c, option2)], false), - // Test whether ordering equivalence works as expected - (vec![(col_d, option1)], true), - (vec![(col_d, option1), (col_b, option1)], true), - (vec![(col_d, option2), (col_b, option1)], false), - ( - vec![(col_e, option2), (col_f, option1), (col_g, option1)], - true, - ), - (vec![(col_e, option2), (col_f, option1)], true), - (vec![(col_e, option1), (col_f, option1)], false), - (vec![(col_e, option2), (col_b, option1)], false), - (vec![(col_e, option1), (col_b, option1)], false), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_d, option1), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option2), - (col_f, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option2), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_d, option2), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option1), - (col_f, option1), - ], - false, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option1), - (col_b, option1), - ], - false, - ), - (vec![(col_d, option1), (col_e, option2)], true), - ]; - - for (cols, expected) in requirements { - let err_msg = format!("Error in test case:{cols:?}"); - let required = cols - .into_iter() - .map(|(col, options)| PhysicalSortExpr { - expr: Arc::new(col.clone()), - options, - }) - .collect::>(); - - let required = Some(&required[..]); - assert_eq!( - ordering_satisfy( - provided, - required, - || eq_properties.clone(), - || ordering_eq_properties.clone(), - ), - expected, - "{err_msg}" - ); - } - Ok(()) - } - - fn convert_to_requirement( - in_data: &[(&Column, Option)], - ) -> Vec { - in_data - .iter() - .map(|(col, options)| { - PhysicalSortRequirement::new(Arc::new((*col).clone()) as _, *options) - }) - .collect::>() - } - - #[test] - fn test_normalize_sort_reqs() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function - let requirements = vec![ - (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), - (vec![(col_a, Some(option2))], vec![(col_a, Some(option2))]), - (vec![(col_a, None)], vec![(col_a, Some(option1))]), - // Test whether equivalence works as expected - (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), - (vec![(col_c, None)], vec![(col_a, Some(option1))]), - // Test whether ordering equivalence works as expected - ( - vec![(col_d, Some(option1)), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_d, None), (col_b, None)], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_e, Some(option2)), (col_f, Some(option1))], - vec![(col_a, Some(option1))], - ), - // We should be able to normalize in compatible requirements also (not exactly equal) - ( - vec![(col_e, Some(option2)), (col_f, None)], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_e, None), (col_f, None)], - vec![(col_a, Some(option1))], - ), - ]; - - let (_test_schema, eq_properties, ordering_eq_properties) = create_test_params()?; - for (reqs, expected_normalized) in requirements.into_iter() { - let req = convert_to_requirement(&reqs); - let expected_normalized = convert_to_requirement(&expected_normalized); - - assert_eq!( - normalize_sort_requirements( - &req, - &eq_properties, - &ordering_eq_properties, - ), - expected_normalized - ); - } - Ok(()) + assert_eq!(get_indices_of_exprs_strict(&list1, &list2), vec![2, 0, 1]); + assert_eq!(get_indices_of_exprs_strict(&list2, &list1), vec![1, 2, 0]); } #[test] @@ -1376,174 +548,6 @@ mod tests { assert_eq!(actual.as_ref(), expected.as_any()); } - #[test] - fn test_normalize_expr_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let _col_d = &Column::new("d", 3); - let _col_e = &Column::new("e", 4); - // Assume that column a and c are aliases. - let (_test_schema, eq_properties, _ordering_eq_properties) = - create_test_params()?; - - let col_a_expr = Arc::new(col_a.clone()) as Arc; - let col_b_expr = Arc::new(col_b.clone()) as Arc; - let col_c_expr = Arc::new(col_c.clone()) as Arc; - // Test cases for equivalence normalization, - // First entry in the tuple is argument, second entry is expected result after normalization. - let expressions = vec![ - // Normalized version of the column a and c should go to a (since a is head) - (&col_a_expr, &col_a_expr), - (&col_c_expr, &col_a_expr), - // Cannot normalize column b - (&col_b_expr, &col_b_expr), - ]; - for (expr, expected_eq) in expressions { - assert!( - expected_eq.eq(&eq_properties.normalize_expr(expr.clone())), - "error in test: expr: {expr:?}" - ); - } - - Ok(()) - } - - #[test] - fn test_normalize_sort_requirement_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let _col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let _col_e = &Column::new("e", 4); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - // Assume that column a and c are aliases. - let (_test_schema, eq_properties, _ordering_eq_properties) = - create_test_params()?; - - // Test cases for equivalence normalization - // First entry in the tuple is PhysicalExpr, second entry is its ordering, third entry is result after normalization. - let expressions = vec![ - (&col_a, Some(option1), &col_a, Some(option1)), - (&col_c, Some(option1), &col_a, Some(option1)), - (&col_c, None, &col_a, None), - // Cannot normalize column d, since it is not in equivalence properties. - (&col_d, Some(option1), &col_d, Some(option1)), - ]; - for (expr, sort_options, expected_col, expected_options) in - expressions.into_iter() - { - let expected = PhysicalSortRequirement::new( - Arc::new((*expected_col).clone()) as _, - expected_options, - ); - let arg = PhysicalSortRequirement::new( - Arc::new((*expr).clone()) as _, - sort_options, - ); - assert!( - expected.eq(&eq_properties.normalize_sort_requirement(arg.clone())), - "error in test: expr: {expr:?}, sort_options: {sort_options:?}" - ); - } - - Ok(()) - } - - #[test] - fn test_ordering_satisfy_different_lengths() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let test_schema = create_test_schema()?; - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - // Column a and c are aliases. - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - eq_properties.add_equal_conditions((col_a, col_c)); - - // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) - let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); - ordering_eq_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], - &vec![PhysicalSortExpr { - expr: Arc::new(col_e.clone()), - options: option1, - }], - )); - let sort_req_a = PhysicalSortExpr { - expr: Arc::new((col_a).clone()) as _, - options: option1, - }; - let sort_req_b = PhysicalSortExpr { - expr: Arc::new((col_b).clone()) as _, - options: option1, - }; - let sort_req_c = PhysicalSortExpr { - expr: Arc::new((col_c).clone()) as _, - options: option1, - }; - let sort_req_d = PhysicalSortExpr { - expr: Arc::new((col_d).clone()) as _, - options: option1, - }; - let sort_req_e = PhysicalSortExpr { - expr: Arc::new((col_e).clone()) as _, - options: option1, - }; - - assert!(ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC, d ASC - &[sort_req_a.clone(), sort_req_b.clone(), sort_req_d.clone()], - // After normalization would be a ASC, b ASC, d ASC - &[ - sort_req_c.clone(), - sort_req_b.clone(), - sort_req_a.clone(), - sort_req_d.clone(), - sort_req_e.clone(), - ], - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )); - - assert!(!ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC - &[sort_req_a.clone(), sort_req_b.clone()], - // After normalization would be a ASC, b ASC, d ASC - &[ - sort_req_c.clone(), - sort_req_b.clone(), - sort_req_a.clone(), - sort_req_d.clone(), - sort_req_e.clone(), - ], - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )); - - assert!(!ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC, d ASC - &[sort_req_a.clone(), sort_req_b.clone(), sort_req_d.clone()], - // After normalization would be a ASC, d ASC, b ASC - &[sort_req_c, sort_req_d, sort_req_a, sort_req_b, sort_req_e,], - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )); - - Ok(()) - } - #[test] fn test_collect_columns() -> Result<()> { let expr1 = Arc::new(Column::new("col1", 2)) as _; @@ -1635,7 +639,15 @@ mod tests { let sort_options = SortOptions::default(); let sort_options_not = SortOptions::default().not(); - let provided_sorts = [ + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ]); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1644,45 +656,38 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); + ]]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &equal_properties, - &ordering_equal_properties, - ), - Some((vec![sort_options_not, sort_options], vec![0, 1])) + result, + vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ] ); - // required columns are provided in the equivalence classes - let provided_sorts = [PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; let schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let mut ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); - ordering_equal_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([ + vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, }], - &vec![ + vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1692,19 +697,36 @@ mod tests { options: sort_options, }, ], - )); + ]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &equal_properties, - &ordering_equal_properties, - ), - Some((vec![sort_options_not, sort_options], vec![0, 1])) + result, + vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ] ); + let required_columns = [ + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("a", 0)) as _, + ]; + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); + // not satisfied orders - let provided_sorts = [ + eq_properties.add_new_orderings([vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1717,25 +739,9 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); - assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &equal_properties, - &ordering_equal_properties, - ), - None - ); + ]]); + let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0]); Ok(()) } @@ -1749,39 +755,39 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let mut expected_oeq = OrderingEquivalenceProperties::new(Arc::new(schema)); + let col_a_expr = col("a", &schema)?; + let col_b_expr = col("b", &schema)?; + let col_c_expr = col("c", &schema)?; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + + eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr); + let others = vec![ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), + options: sort_options, + }], + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), + options: sort_options, + }], + ]; + eq_properties.add_new_orderings(others); - equal_properties - .add_equal_conditions((&Column::new("a", 0), &Column::new("c", 2))); - let head = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: sort_options, - }]; - let others = vec![vec![PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }]]; - let oeq_class = OrderingEquivalentClass::new(head, others); - - expected_oeq.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), + let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); + expected_eqs.add_new_orderings([ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), options: sort_options, }], - &vec![PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), options: sort_options, }], - )); - - let normalized_oeq_class = - oeq_class.normalize_with_equivalence_properties(&equal_properties); - let expected = expected_oeq.oeq_class().unwrap(); - assert!( - normalized_oeq_class.head().eq(expected.head()) - && normalized_oeq_class.others().eq(expected.others()) - ); + ]); + + let oeq_class = eq_properties.oeq_class().clone(); + let expected = expected_eqs.oeq_class(); + assert!(oeq_class.eq(expected)); Ok(()) } @@ -1793,27 +799,37 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let orderings = find_orderings_of_exprs( - &[ - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - ], - Some(&[PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }]), - EquivalenceProperties::new(Arc::new(schema.clone())), - OrderingEquivalenceProperties::new(Arc::new(schema.clone())), - )?; + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + let ordering = vec![PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }]; + eq_properties.add_new_orderings([ordering]); + let projection_mapping = vec![ + ( + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("b_new", 0)) as _, + ), + ( + Arc::new(Column::new("a", 0)) as _, + Arc::new(Column::new("a_new", 1)) as _, + ), + ]; + let projection_schema = Arc::new(Schema::new(vec![ + Field::new("b_new", DataType::Int32, true), + Field::new("a_new", DataType::Int32, true), + ])); + let orderings = eq_properties + .project(&projection_mapping, projection_schema) + .oeq_class() + .output_ordering() + .unwrap_or_default(); assert_eq!( - vec![ - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 0)), - options: SortOptions::default(), - }), - None, - ], + vec![PhysicalSortExpr { + expr: Arc::new(Column::new("b_new", 0)), + options: SortOptions::default(), + }], orderings ); @@ -1822,105 +838,24 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let orderings = find_orderings_of_exprs( - &[ - (Arc::new(Column::new("c", 2)), "c_new".to_string()), - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - ], - Some(&[]), - EquivalenceProperties::new(Arc::new(schema.clone())), - OrderingEquivalenceProperties::new(Arc::new(schema)), - )?; - - assert_eq!(vec![None, None], orderings); - - Ok(()) - } - - #[test] - fn test_find_orderings_of_exprs() -> Result<()> { - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - ]); - - let mut eq = EquivalenceProperties::new(Arc::new(schema.clone())); - let col_a = &col("a", &schema)?; - let col_b = &col("b", &schema)?; - let col_c = &col("c", &schema)?; - let col_d = &col("d", &schema)?; - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - // b=a (e.g they are aliases) - eq.add_equal_conditions((&Column::new("b", 1), &Column::new("a", 0))); - let mut oeq = OrderingEquivalenceProperties::new(Arc::new(schema.clone())); - // [b ASC], [d ASC] - oeq.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: col_b.clone(), - options: option_asc, - }], - &vec![PhysicalSortExpr { - expr: col_d.clone(), - options: option_asc, - }], - )); - - let orderings = find_orderings_of_exprs( - &[ - // d + b - ( - Arc::new(BinaryExpr::new( - col_d.clone(), - Operator::Plus, - col_b.clone(), - )), - "d+b".to_string(), - ), - // b as b_new - (col_b.clone(), "b_new".to_string()), - // a as a_new - (col_a.clone(), "a_new".to_string()), - // a + c - ( - Arc::new(BinaryExpr::new( - col_a.clone(), - Operator::Plus, - col_c.clone(), - )), - "a+c".to_string(), - ), - ], - Some(&[PhysicalSortExpr { - expr: col_b.clone(), - options: option_asc, - }]), - eq, - oeq, - )?; - - assert_eq!( - vec![ - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("d+b", 0)), - options: option_asc, - }), - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 1)), - options: option_asc, - }), - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("a_new", 2)), - options: option_asc, - }), - None, - ], - orderings - ); + let eq_properties = EquivalenceProperties::new(Arc::new(schema)); + let projection_mapping = vec![ + ( + Arc::new(Column::new("c", 2)) as _, + Arc::new(Column::new("c_new", 0)) as _, + ), + ( + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("b_new", 1)) as _, + ), + ]; + let projection_schema = Arc::new(Schema::new(vec![ + Field::new("c_new", DataType::Int32, true), + Field::new("b_new", DataType::Int32, true), + ])); + let projected = eq_properties.project(&projection_mapping, projection_schema); + // After projection there is no ordering. + assert!(projected.oeq_class().output_ordering().is_none()); Ok(()) } diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index a00d32e201fb..665ceb70d658 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -21,22 +21,19 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use super::BuiltInWindowFunctionExpr; -use super::WindowExpr; -use crate::equivalence::OrderingEquivalenceBuilder; +use super::{BuiltInWindowFunctionExpr, WindowExpr}; use crate::expressions::PhysicalSortExpr; -use crate::utils::{convert_to_expr, get_indices_of_matching_exprs}; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; + use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::window_state::WindowAggState; -use datafusion_expr::window_state::WindowFrameContext; +use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; /// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. @@ -75,16 +72,12 @@ impl BuiltInWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings EquivalenceProperties>( - &self, - builder: &mut OrderingEquivalenceBuilder, - equal_properties: F, - ) { - let schema = builder.schema(); + pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { + let schema = eq_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - builder.add_equal_conditions(vec![fn_res_ordering]); + eq_properties.add_new_orderings([vec![fn_res_ordering]]); } else { // If we have a PARTITION BY, built-in functions can not introduce // a global ordering unless the existing ordering is compatible @@ -92,23 +85,11 @@ impl BuiltInWindowExpr { // expressions and existing ordering expressions are equal (w.r.t. // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. - let existing_ordering = builder.existing_ordering(); - let existing_ordering_exprs = convert_to_expr(existing_ordering); - // Get indices of the PARTITION BY expressions among input ordering expressions: - let pb_indices = get_indices_of_matching_exprs( - &self.partition_by, - &existing_ordering_exprs, - equal_properties, - ); - // Existing ordering should match exactly with PARTITION BY expressions. - // There should be no missing/extra entries in the existing ordering. - // Otherwise, prefixing wouldn't work. - if pb_indices.len() == self.partition_by.len() - && pb_indices.len() == existing_ordering.len() - { - let mut new_ordering = existing_ordering.to_vec(); - new_ordering.push(fn_res_ordering); - builder.add_equal_conditions(new_ordering); + let (mut ordering, _) = + eq_properties.find_longest_permutation(&self.partition_by); + if ordering.len() == self.partition_by.len() { + ordering.push(fn_res_ordering); + eq_properties.add_new_orderings([ordering]); } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index da152a6264af..fd2dc69e1f3d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -18,7 +18,6 @@ //! Aggregates functionalities use std::any::Any; -use std::collections::HashMap; use std::sync::Arc; use super::DisplayAs; @@ -26,9 +25,13 @@ use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; +use crate::common::calculate_projection_mapping; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::windows::{ + get_ordered_partition_by_indices, get_window_mode, PartitionSearchMode, +}; use crate::{ - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; @@ -36,24 +39,18 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; -use datafusion_physical_expr::utils::{ - convert_to_expr, get_finer_ordering, get_indices_of_matching_exprs, - ordering_satisfy_requirement_concrete, -}; use datafusion_physical_expr::{ aggregate::is_order_sensitive, - equivalence::project_equivalence_properties, - expressions::{Column, Max, Min}, - normalize_out_expr_with_columns_map, physical_exprs_contains, reverse_order_bys, - AggregateExpr, LexOrdering, LexOrderingReq, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + equivalence::collapse_lex_req, + expressions::{Column, Max, Min, UnKnownColumn}, + physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, + LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; -use itertools::Itertools; +use itertools::{izip, Itertools}; mod group_values; mod no_grouping; @@ -271,18 +268,6 @@ impl From for SendableRecordBatchStream { } } -/// This object encapsulates ordering-related information on GROUP BY columns. -#[derive(Debug, Clone)] -pub(crate) struct AggregationOrdering { - /// Specifies whether the GROUP BY columns are partially or fully ordered. - mode: GroupByOrderMode, - /// Stores indices such that when we iterate with these indices, GROUP BY - /// expressions match input ordering. - order_indices: Vec, - /// Actual ordering information of the GROUP BY columns. - ordering: LexOrdering, -} - /// Hash aggregate execution plan #[derive(Debug)] pub struct AggregateExec { @@ -308,90 +293,15 @@ 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 columns map used to normalize out expressions like Partitioning and PhysicalSortExpr - /// The key is the column from the input schema and the values are the columns from the output schema - columns_map: HashMap>, - /// Execution Metrics + /// The mapping used to normalize expressions like Partitioning and + /// PhysicalSortExpr. The key is the expression from the input schema + /// and the value is the expression from the output schema. + projection_mapping: Vec<(Arc, Arc)>, + /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Stores mode and output ordering information for the `AggregateExec`. - aggregation_ordering: Option, - required_input_ordering: Option, -} - -/// Calculates the working mode for `GROUP BY` queries. -/// - If no GROUP BY expression has an ordering, returns `None`. -/// - If some GROUP BY expressions have an ordering, returns `Some(GroupByOrderMode::PartiallyOrdered)`. -/// - If all GROUP BY expressions have orderings, returns `Some(GroupByOrderMode::Ordered)`. -fn get_working_mode( - input: &Arc, - group_by: &PhysicalGroupBy, -) -> Option<(GroupByOrderMode, Vec)> { - if !group_by.is_single() { - // We do not currently support streaming execution if we have more - // than one group (e.g. we have grouping sets). - return None; - }; - - let output_ordering = input.output_ordering().unwrap_or(&[]); - // Since direction of the ordering is not important for GROUP BY columns, - // we convert PhysicalSortExpr to PhysicalExpr in the existing ordering. - let ordering_exprs = convert_to_expr(output_ordering); - let groupby_exprs = group_by.input_exprs(); - // Find where each expression of the GROUP BY clause occurs in the existing - // ordering (if it occurs): - let mut ordered_indices = - get_indices_of_matching_exprs(&groupby_exprs, &ordering_exprs, || { - input.equivalence_properties() - }); - ordered_indices.sort(); - // Find out how many expressions of the existing ordering define ordering - // for expressions in the GROUP BY clause. For example, if the input is - // ordered by a, b, c, d and we group by b, a, d; the result below would be. - // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. - let first_n = longest_consecutive_prefix(ordered_indices); - if first_n == 0 { - // No GROUP by columns are ordered, we can not do streaming execution. - return None; - } - let ordered_exprs = ordering_exprs[0..first_n].to_vec(); - // Find indices for the GROUP BY expressions such that when we iterate with - // these indices, we would match existing ordering. For the example above, - // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the - // GROUP BY expressions b, a, d match input ordering. - let ordered_group_by_indices = - get_indices_of_matching_exprs(&ordered_exprs, &groupby_exprs, || { - input.equivalence_properties() - }); - Some(if first_n == group_by.expr.len() { - (GroupByOrderMode::FullyOrdered, ordered_group_by_indices) - } else { - (GroupByOrderMode::PartiallyOrdered, ordered_group_by_indices) - }) -} - -/// This function gathers the ordering information for the GROUP BY columns. -fn calc_aggregation_ordering( - input: &Arc, - group_by: &PhysicalGroupBy, -) -> Option { - get_working_mode(input, group_by).map(|(mode, order_indices)| { - let existing_ordering = input.output_ordering().unwrap_or(&[]); - let out_group_expr = group_by.output_exprs(); - // Calculate output ordering information for the operator: - let out_ordering = order_indices - .iter() - .zip(existing_ordering) - .map(|(idx, input_col)| PhysicalSortExpr { - expr: out_group_expr[*idx].clone(), - options: input_col.options, - }) - .collect::>(); - AggregationOrdering { - mode, - order_indices, - ordering: out_ordering, - } - }) + required_input_ordering: Option, + partition_search_mode: PartitionSearchMode, + output_ordering: Option, } /// This function returns the ordering requirement of the first non-reversible @@ -420,46 +330,57 @@ fn get_init_req( /// This function gets the finest ordering requirement among all the aggregation /// functions. If requirements are conflicting, (i.e. we can not compute the /// aggregations in a single [`AggregateExec`]), the function returns an error. -fn get_finest_requirement< - F: Fn() -> EquivalenceProperties, - F2: Fn() -> OrderingEquivalenceProperties, ->( +fn get_finest_requirement( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], - eq_properties: F, - ordering_eq_properties: F2, + eq_properties: &EquivalenceProperties, ) -> Result> { + // First, we check if all the requirements are satisfied by the existing + // ordering. If so, we return `None` to indicate this. + let mut all_satisfied = true; + for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { + if eq_properties.ordering_satisfy(fn_req.as_deref().unwrap_or(&[])) { + continue; + } + if let Some(reverse) = aggr_expr.reverse_expr() { + let reverse_req = fn_req.as_ref().map(|item| reverse_order_bys(item)); + if eq_properties.ordering_satisfy(reverse_req.as_deref().unwrap_or(&[])) { + // We need to update `aggr_expr` with its reverse since only its + // reverse requirement is compatible with the existing requirements: + *aggr_expr = reverse; + *fn_req = reverse_req; + continue; + } + } + // Requirement is not satisfied: + all_satisfied = false; + } + if all_satisfied { + // All of the requirements are already satisfied. + return Ok(None); + } let mut finest_req = get_init_req(aggr_expr, order_by_expr); for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - let fn_req = if let Some(fn_req) = fn_req { - fn_req - } else { + let Some(fn_req) = fn_req else { continue; }; + if let Some(finest_req) = &mut finest_req { - if let Some(finer) = get_finer_ordering( - finest_req, - fn_req, - &eq_properties, - &ordering_eq_properties, - ) { - *finest_req = finer.to_vec(); + if let Some(finer) = eq_properties.get_finer_ordering(finest_req, fn_req) { + *finest_req = finer; continue; } // If an aggregate function is reversible, analyze whether its reverse // direction is compatible with existing requirements: if let Some(reverse) = aggr_expr.reverse_expr() { let fn_req_reverse = reverse_order_bys(fn_req); - if let Some(finer) = get_finer_ordering( - finest_req, - &fn_req_reverse, - &eq_properties, - &ordering_eq_properties, - ) { + if let Some(finer) = + eq_properties.get_finer_ordering(finest_req, &fn_req_reverse) + { // We need to update `aggr_expr` with its reverse, since only its // reverse requirement is compatible with existing requirements: *aggr_expr = reverse; - *finest_req = finer.to_vec(); + *finest_req = finer; *fn_req = fn_req_reverse; continue; } @@ -477,113 +398,46 @@ fn get_finest_requirement< Ok(finest_req) } -/// Calculate the required input ordering for the [`AggregateExec`] by considering -/// ordering requirements of order-sensitive aggregation functions. -fn calc_required_input_ordering( +/// Calculates search_mode for the aggregation +fn get_aggregate_search_mode( + group_by: &PhysicalGroupBy, input: &Arc, - aggr_exprs: &mut [Arc], - order_by_exprs: &mut [Option], - aggregator_reqs: LexOrderingReq, - aggregator_reverse_reqs: Option, - aggregation_ordering: &mut Option, - mode: &AggregateMode, -) -> Result> { - let mut required_input_ordering = vec![]; - // Boolean shows that whether `required_input_ordering` stored comes from - // `aggregator_reqs` or `aggregator_reverse_reqs` - let mut reverse_req = false; - // If reverse aggregator is None, there is no way to run aggregators in reverse mode. Hence ignore it during analysis - let aggregator_requirements = - if let Some(aggregator_reverse_reqs) = aggregator_reverse_reqs { - // If existing ordering doesn't satisfy requirement, we should do calculations - // on naive requirement (by convention, otherwise the final plan will be unintuitive), - // even if reverse ordering is possible. - // Hence, while iterating consider naive requirement last, by this way - // we prioritize naive requirement over reverse requirement, when - // reverse requirement is not helpful with removing SortExec from the plan. - vec![(true, aggregator_reverse_reqs), (false, aggregator_reqs)] - } else { - vec![(false, aggregator_reqs)] - }; - for (is_reverse, aggregator_requirement) in aggregator_requirements.into_iter() { - if let Some(AggregationOrdering { - // If the mode is FullyOrdered or PartiallyOrdered (i.e. we are - // running with bounded memory, without breaking the pipeline), - // then we append the aggregator ordering requirement to the existing - // ordering. This way, we can still run with bounded memory. - mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, - order_indices, - .. - }) = aggregation_ordering - { - // Get the section of the input ordering that enables us to run in - // FullyOrdered or PartiallyOrdered modes: - let requirement_prefix = - if let Some(existing_ordering) = input.output_ordering() { - &existing_ordering[0..order_indices.len()] - } else { - &[] - }; - let mut requirement = - PhysicalSortRequirement::from_sort_exprs(requirement_prefix.iter()); - for req in aggregator_requirement { - // Final and FinalPartitioned modes don't enforce ordering - // requirements since order-sensitive aggregators handle such - // requirements during merging. - if mode.is_first_stage() - && requirement.iter().all(|item| req.expr.ne(&item.expr)) - { - requirement.push(req); - } - } - required_input_ordering = requirement; - } else if mode.is_first_stage() { - required_input_ordering = aggregator_requirement; - } - // Keep track of the direction from which required_input_ordering is constructed: - reverse_req = is_reverse; - // If all the order-sensitive aggregate functions are reversible (e.g. all the - // order-sensitive aggregators are either FIRST_VALUE or LAST_VALUE), then we can - // run aggregate expressions either in the given required ordering, (i.e. finest - // requirement that satisfies every aggregate function requirement) or its reverse - // (opposite) direction. We analyze these two possibilities, and use the version that - // satisfies existing ordering. This enables us to avoid an extra sort step in the final - // plan. If neither version satisfies the existing ordering, we use the given ordering - // requirement. In short, if running aggregators in reverse order help us to avoid a - // sorting step, we do so. Otherwise, we use the aggregators as is. - let existing_ordering = input.output_ordering().unwrap_or(&[]); - if ordering_satisfy_requirement_concrete( - existing_ordering, - &required_input_ordering, - || input.equivalence_properties(), - || input.ordering_equivalence_properties(), - ) { - break; - } + aggr_expr: &mut [Arc], + order_by_expr: &mut [Option], + ordering_req: &mut Vec, +) -> Result { + let groupby_exprs = group_by + .expr + .iter() + .map(|(item, _)| item.clone()) + .collect::>(); + let mut partition_search_mode = PartitionSearchMode::Linear; + if !group_by.is_single() || groupby_exprs.is_empty() { + return Ok(partition_search_mode); } - // If `required_input_ordering` is constructed using the reverse requirement, we - // should reverse each `aggr_expr` in order to correctly calculate their results - // in reverse order. - if reverse_req { - aggr_exprs - .iter_mut() - .zip(order_by_exprs.iter_mut()) - .map(|(aggr_expr, ob_expr)| { - if is_order_sensitive(aggr_expr) { - if let Some(reverse) = aggr_expr.reverse_expr() { - *aggr_expr = reverse; - *ob_expr = ob_expr.as_ref().map(|obs| reverse_order_bys(obs)); + + if let Some((should_reverse, mode)) = + get_window_mode(&groupby_exprs, ordering_req, input)? + { + let all_reversible = aggr_expr + .iter() + .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); + if should_reverse && all_reversible { + izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( + |(aggr, order_by)| { + if let Some(reverse) = aggr.reverse_expr() { + *aggr = reverse; } else { - return plan_err!( - "Aggregate expression should have a reverse expression" - ); + unreachable!(); } - } - Ok(()) - }) - .collect::>>()?; + *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); + }, + ); + *ordering_req = reverse_order_bys(ordering_req); + } + partition_search_mode = mode; } - Ok((!required_input_ordering.is_empty()).then_some(required_input_ordering)) + Ok(partition_search_mode) } /// Check whether group by expression contains all of the expression inside `requirement` @@ -647,57 +501,50 @@ impl AggregateExec { }) }) .collect::>(); - let mut aggregator_reverse_reqs = None; - // Currently we support order-sensitive aggregation only in `Single` mode. - // For `Final` and `FinalPartitioned` modes, we cannot guarantee they will receive - // data according to ordering requirements. As long as we cannot produce correct result - // in `Final` mode, it is not important to produce correct result in `Partial` mode. - // We only support `Single` mode, where we are sure that output produced is final, and it - // is produced in a single step. - let requirement = get_finest_requirement( &mut aggr_expr, &mut order_by_expr, - || input.equivalence_properties(), - || input.ordering_equivalence_properties(), + &input.equivalence_properties(), )?; - let aggregator_requirement = requirement - .as_ref() - .map(|exprs| PhysicalSortRequirement::from_sort_exprs(exprs.iter())); - let aggregator_reqs = aggregator_requirement.unwrap_or(vec![]); - // If all aggregate expressions are reversible, also consider reverse - // requirement(s). The reason is that existing ordering may satisfy the - // given requirement or its reverse. By considering both, we can generate better plans. - if aggr_expr - .iter() - .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()) - { - aggregator_reverse_reqs = requirement.map(|reqs| { - PhysicalSortRequirement::from_sort_exprs(reverse_order_bys(&reqs).iter()) - }); - } - - // construct a map from the input columns to the output columns of the Aggregation - let mut columns_map: HashMap> = HashMap::new(); - for (expression, name) in group_by.expr.iter() { - if let Some(column) = expression.as_any().downcast_ref::() { - let new_col_idx = schema.index_of(name)?; - let entry = columns_map.entry(column.clone()).or_default(); - entry.push(Column::new(name, new_col_idx)); - }; - } - - let mut aggregation_ordering = calc_aggregation_ordering(&input, &group_by); - let required_input_ordering = calc_required_input_ordering( + let mut ordering_req = requirement.unwrap_or(vec![]); + let partition_search_mode = get_aggregate_search_mode( + &group_by, &input, &mut aggr_expr, &mut order_by_expr, - aggregator_reqs, - aggregator_reverse_reqs, - &mut aggregation_ordering, - &mode, + &mut ordering_req, )?; + // Get GROUP BY expressions: + let groupby_exprs = group_by.input_exprs(); + // If existing ordering satisfies a prefix of the GROUP BY expressions, + // prefix requirements with this section. In this case, aggregation will + // work more efficiently. + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let mut new_requirement = indices + .into_iter() + .map(|idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + // Postfix ordering requirement of the aggregation to the requirement. + let req = PhysicalSortRequirement::from_sort_exprs(&ordering_req); + new_requirement.extend(req); + new_requirement = collapse_lex_req(new_requirement); + + // construct a map from the input expression to the output expression of the Aggregation group by + let projection_mapping = + calculate_projection_mapping(&group_by.expr, &input.schema())?; + + let required_input_ordering = + (!new_requirement.is_empty()).then_some(new_requirement); + + let aggregate_eqs = input + .equivalence_properties() + .project(&projection_mapping, schema.clone()); + let output_ordering = aggregate_eqs.oeq_class().output_ordering(); + Ok(AggregateExec { mode, group_by, @@ -707,11 +554,12 @@ impl AggregateExec { input, schema, input_schema, - columns_map, + projection_mapping, metrics: ExecutionPlanMetricsSet::new(), - aggregation_ordering, required_input_ordering, limit: None, + partition_search_mode, + output_ordering, }) } @@ -870,8 +718,8 @@ impl DisplayAs for AggregateExec { write!(f, ", lim=[{limit}]")?; } - if let Some(aggregation_ordering) = &self.aggregation_ordering { - write!(f, ", ordering_mode={:?}", aggregation_ordering.mode)?; + if self.partition_search_mode != PartitionSearchMode::Linear { + write!(f, ", ordering_mode={:?}", self.partition_search_mode)?; } } } @@ -893,23 +741,28 @@ impl ExecutionPlan for AggregateExec { 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| { - normalize_out_expr_with_columns_map(expr, &self.columns_map) + 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 + .collect(); + return Partitioning::Hash(normalized_exprs, part); } - } else { - // Final Aggregation's output partitioning is the same as its real input - input_partition } + // Final Aggregation's output partitioning is the same as its real input + input_partition } /// Specifies whether this plan generates an infinite stream of records. @@ -917,7 +770,7 @@ impl ExecutionPlan for AggregateExec { /// infinite, returns an error to indicate this. fn unbounded_output(&self, children: &[bool]) -> Result { if children[0] { - if self.aggregation_ordering.is_none() { + if self.partition_search_mode == PartitionSearchMode::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." @@ -931,9 +784,7 @@ impl ExecutionPlan for AggregateExec { } fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.aggregation_ordering - .as_ref() - .map(|item: &AggregationOrdering| item.ordering.as_slice()) + self.output_ordering.as_deref() } fn required_input_distribution(&self) -> Vec { @@ -950,18 +801,14 @@ impl ExecutionPlan for AggregateExec { } } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![self.required_input_ordering.clone()] } fn equivalence_properties(&self) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(self.schema()); - project_equivalence_properties( - self.input.equivalence_properties(), - &self.columns_map, - &mut new_properties, - ); - new_properties + self.input + .equivalence_properties() + .project(&self.projection_mapping, self.schema()) } fn children(&self) -> Vec> { @@ -1287,19 +1134,21 @@ pub(crate) fn evaluate_group_by( #[cfg(test)] mod tests { + use std::any::Any; + use std::sync::Arc; + use std::task::{Context, Poll}; + use super::*; - use crate::aggregates::GroupByOrderMode::{FullyOrdered, PartiallyOrdered}; use crate::aggregates::{ - get_finest_requirement, get_working_mode, AggregateExec, AggregateMode, - PhysicalGroupBy, + get_finest_requirement, AggregateExec, AggregateMode, PhysicalGroupBy, }; use crate::coalesce_batches::CoalesceBatchesExec; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::common; use crate::expressions::{col, Avg}; use crate::memory::MemoryExec; + use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::{assert_is_pending, mem_exec}; use crate::{ DisplayAs, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -1313,20 +1162,15 @@ mod tests { assert_batches_eq, assert_batches_sorted_eq, internal_err, DataFusionError, Result, ScalarValue, }; + use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Column, Count, FirstValue, LastValue, Median, + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, }; use datafusion_physical_expr::{ - AggregateExpr, EquivalenceProperties, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, + AggregateExpr, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, }; - use std::any::Any; - use std::sync::Arc; - use std::task::{Context, Poll}; - - use datafusion_execution::config::SessionConfig; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -1341,80 +1185,6 @@ mod tests { Ok(schema) } - /// make PhysicalSortExpr with default options - fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { - sort_expr_options(name, schema, SortOptions::default()) - } - - /// PhysicalSortExpr with specified options - fn sort_expr_options( - name: &str, - schema: &Schema, - options: SortOptions, - ) -> PhysicalSortExpr { - PhysicalSortExpr { - expr: col(name, schema).unwrap(), - options, - } - } - - #[tokio::test] - async fn test_get_working_mode() -> Result<()> { - let test_schema = create_test_schema()?; - // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST - // Column d, e is not ordered. - let sort_exprs = vec![ - sort_expr("a", &test_schema), - sort_expr("b", &test_schema), - sort_expr("c", &test_schema), - ]; - let input = mem_exec(1).with_sort_information(vec![sort_exprs]); - let input = Arc::new(input) as _; - - // test cases consists of vector of tuples. Where each tuple represents a single test case. - // First field in the tuple is Vec where each element in the vector represents GROUP BY columns - // For instance `vec!["a", "b"]` corresponds to GROUP BY a, b - // Second field in the tuple is Option, which corresponds to expected algorithm mode. - // None represents that existing ordering is not sufficient to run executor with any one of the algorithms - // (We need to add SortExec to be able to run it). - // Some(GroupByOrderMode) represents, we can run algorithm with existing ordering; and algorithm should work in - // GroupByOrderMode. - let test_cases = vec![ - (vec!["a"], Some((FullyOrdered, vec![0]))), - (vec!["b"], None), - (vec!["c"], None), - (vec!["b", "a"], Some((FullyOrdered, vec![1, 0]))), - (vec!["c", "b"], None), - (vec!["c", "a"], Some((PartiallyOrdered, vec![1]))), - (vec!["c", "b", "a"], Some((FullyOrdered, vec![2, 1, 0]))), - (vec!["d", "a"], Some((PartiallyOrdered, vec![1]))), - (vec!["d", "b"], None), - (vec!["d", "c"], None), - (vec!["d", "b", "a"], Some((PartiallyOrdered, vec![2, 1]))), - (vec!["d", "c", "b"], None), - (vec!["d", "c", "a"], Some((PartiallyOrdered, vec![2]))), - ( - vec!["d", "c", "b", "a"], - Some((PartiallyOrdered, vec![3, 2, 1])), - ), - ]; - for (case_idx, test_case) in test_cases.iter().enumerate() { - let (group_by_columns, expected) = &test_case; - let mut group_by_exprs = vec![]; - for col_name in group_by_columns { - group_by_exprs.push((col(col_name, &test_schema)?, col_name.to_string())); - } - let group_bys = PhysicalGroupBy::new_single(group_by_exprs); - let res = get_working_mode(&input, &group_bys); - assert_eq!( - res, *expected, - "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" - ); - } - - Ok(()) - } - /// some mock data to aggregates fn some_data() -> (Arc, Vec) { // define a schema. @@ -2268,69 +2038,72 @@ mod tests { descending: true, nulls_first: true, }; - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - let col_a = Column::new("a", 0); - let col_b = Column::new("b", 1); - let col_c = Column::new("c", 2); - let col_d = Column::new("d", 3); - eq_properties.add_equal_conditions((&col_a, &col_b)); - let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); - ordering_eq_properties.add_equal_conditions(( - &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()) as _, - options: options1, - }], - &vec![PhysicalSortExpr { - expr: Arc::new(col_c.clone()) as _, - options: options2, - }], - )); + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let mut eq_properties = EquivalenceProperties::new(test_schema); + // Columns a and b are equal. + eq_properties.add_equal_conditions(col_a, col_b); + // Aggregate requirements are + // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively let mut order_by_exprs = vec![ None, Some(vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: options1, - }]), - Some(vec![PhysicalSortExpr { - expr: Arc::new(col_b.clone()), + expr: col_a.clone(), options: options1, }]), - Some(vec![PhysicalSortExpr { - expr: Arc::new(col_c), - options: options2, - }]), Some(vec![ PhysicalSortExpr { - expr: Arc::new(col_a.clone()), + expr: col_a.clone(), options: options1, }, PhysicalSortExpr { - expr: Arc::new(col_d), + expr: col_b.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]), + Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_b.clone(), options: options1, }, ]), // Since aggregate expression is reversible (FirstValue), we should be able to resolve below // contradictory requirement by reversing it. Some(vec![PhysicalSortExpr { - expr: Arc::new(col_b.clone()), + expr: col_b.clone(), options: options2, }]), ]; + let common_requirement = Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]); let aggr_expr = Arc::new(FirstValue::new( - Arc::new(col_a.clone()), + col_a.clone(), "first1", DataType::Int32, vec![], vec![], )) as _; let mut aggr_exprs = vec![aggr_expr; order_by_exprs.len()]; - let res = get_finest_requirement( - &mut aggr_exprs, - &mut order_by_exprs, - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )?; - assert_eq!(res, order_by_exprs[4]); + let res = + get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, &eq_properties)?; + assert_eq!(res, common_requirement); Ok(()) } } diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index f0b49872b1c5..f72d2f06e459 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -18,13 +18,12 @@ use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; -use datafusion_physical_expr::EmitTo; - -use super::{AggregationOrdering, GroupByOrderMode}; +use datafusion_physical_expr::{EmitTo, PhysicalSortExpr}; mod full; mod partial; +use crate::windows::PartitionSearchMode; pub(crate) use full::GroupOrderingFull; pub(crate) use partial::GroupOrderingPartial; @@ -43,24 +42,19 @@ impl GroupOrdering { /// Create a `GroupOrdering` for the the specified ordering pub fn try_new( input_schema: &Schema, - ordering: &AggregationOrdering, + mode: &PartitionSearchMode, + ordering: &[PhysicalSortExpr], ) -> Result { - let AggregationOrdering { - mode, - order_indices, - ordering, - } = ordering; - - Ok(match mode { - GroupByOrderMode::PartiallyOrdered => { - let partial = - GroupOrderingPartial::try_new(input_schema, order_indices, ordering)?; - GroupOrdering::Partial(partial) + match mode { + PartitionSearchMode::Linear => Ok(GroupOrdering::None), + PartitionSearchMode::PartiallySorted(order_indices) => { + GroupOrderingPartial::try_new(input_schema, order_indices, ordering) + .map(GroupOrdering::Partial) } - GroupByOrderMode::FullyOrdered => { - GroupOrdering::Full(GroupOrderingFull::new()) + PartitionSearchMode::Sorted => { + Ok(GroupOrdering::Full(GroupOrderingFull::new())) } - }) + } } // How many groups be emitted, or None if no data can be emitted diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index f1eece4315e4..7cee4a3e7cfc 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -17,17 +17,10 @@ //! Hash aggregation -use datafusion_physical_expr::{ - AggregateExpr, EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, PhysicalSortExpr, -}; -use log::debug; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; -use futures::ready; -use futures::stream::{Stream, StreamExt}; - use crate::aggregates::group_values::{new_group_values, GroupValues}; use crate::aggregates::order::GroupOrderingFull; use crate::aggregates::{ @@ -39,8 +32,9 @@ use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::sorts::sort::{read_spill_as_stream, sort_batch}; use crate::sorts::streaming_merge; use crate::stream::RecordBatchStreamAdapter; -use crate::{aggregates, PhysicalExpr}; +use crate::{aggregates, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; + use arrow::array::*; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use arrow_schema::SortOptions; @@ -50,7 +44,14 @@ use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::{ + AggregateExpr, EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, PhysicalSortExpr, +}; + +use futures::ready; +use futures::stream::{Stream, StreamExt}; +use log::debug; #[derive(Debug, Clone)] /// This object tracks the aggregation phase (input/output) @@ -321,8 +322,9 @@ impl GroupedHashAggregateStream { let spill_expr = group_schema .fields .into_iter() - .map(|field| PhysicalSortExpr { - expr: col(field.name(), &group_schema).unwrap(), + .enumerate() + .map(|(idx, field)| PhysicalSortExpr { + expr: Arc::new(Column::new(field.name().as_str(), idx)) as _, options: SortOptions::default(), }) .collect(); @@ -331,16 +333,14 @@ impl GroupedHashAggregateStream { let reservation = MemoryConsumer::new(name) .with_can_spill(true) .register(context.memory_pool()); - - let group_ordering = agg - .aggregation_ordering - .as_ref() - .map(|aggregation_ordering| { - GroupOrdering::try_new(&group_schema, aggregation_ordering) - }) - // return error if any - .transpose()? - .unwrap_or(GroupOrdering::None); + let (ordering, _) = agg + .equivalence_properties() + .find_longest_permutation(&agg_group_by.output_exprs()); + let group_ordering = GroupOrdering::try_new( + &group_schema, + &agg.partition_search_mode, + ordering.as_slice(), + )?; let group_values = new_group_values(group_schema)?; timer.done(); @@ -418,8 +418,7 @@ impl Stream for GroupedHashAggregateStream { let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); loop { - let exec_state = self.exec_state.clone(); - match exec_state { + match &self.exec_state { ExecutionState::ReadingInput => { match ready!(self.input.poll_next_unpin(cx)) { // new batch to aggregate @@ -453,14 +452,14 @@ impl Stream for GroupedHashAggregateStream { self.input_done = true; self.group_ordering.input_done(); let timer = elapsed_compute.timer(); - if self.spill_state.spills.is_empty() { + self.exec_state = if self.spill_state.spills.is_empty() { let batch = extract_ok!(self.emit(EmitTo::All, false)); - self.exec_state = ExecutionState::ProducingOutput(batch); + ExecutionState::ProducingOutput(batch) } else { // If spill files exist, stream-merge them. extract_ok!(self.update_merged_stream()); - self.exec_state = ExecutionState::ReadingInput; - } + ExecutionState::ReadingInput + }; timer.done(); } } @@ -468,19 +467,24 @@ impl Stream for GroupedHashAggregateStream { ExecutionState::ProducingOutput(batch) => { // slice off a part of the batch, if needed - let output_batch = if batch.num_rows() <= self.batch_size { - if self.input_done { - self.exec_state = ExecutionState::Done; - } else { - self.exec_state = ExecutionState::ReadingInput - } - batch + let output_batch; + let size = self.batch_size; + (self.exec_state, output_batch) = if batch.num_rows() <= size { + ( + if self.input_done { + ExecutionState::Done + } else { + ExecutionState::ReadingInput + }, + batch.clone(), + ) } else { // output first batch_size rows - let num_remaining = batch.num_rows() - self.batch_size; - let remaining = batch.slice(self.batch_size, num_remaining); - self.exec_state = ExecutionState::ProducingOutput(remaining); - batch.slice(0, self.batch_size) + let size = self.batch_size; + let num_remaining = batch.num_rows() - size; + let remaining = batch.slice(size, num_remaining); + let output = batch.slice(0, size); + (ExecutionState::ProducingOutput(remaining), output) }; return Poll::Ready(Some(Ok( output_batch.record_output(&self.baseline_metrics) diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index df9e8a8a2b8c..09d1ea87ca37 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -27,8 +27,8 @@ use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, Statistics}; use crate::{ - DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, + DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, }; use arrow::datatypes::SchemaRef; @@ -36,7 +36,7 @@ use arrow::error::Result as ArrowResult; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::EquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -138,10 +138,6 @@ impl ExecutionPlan for CoalesceBatchesExec { self.input.equivalence_properties() } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() - } - fn with_new_children( self: Arc, children: Vec>, @@ -228,17 +224,17 @@ impl CoalesceBatchesStream { let _timer = cloned_time.timer(); match input_batch { Poll::Ready(x) => match x { - Some(Ok(ref batch)) => { + Some(Ok(batch)) => { if batch.num_rows() >= self.target_batch_size && self.buffer.is_empty() { - return Poll::Ready(Some(Ok(batch.clone()))); + return Poll::Ready(Some(Ok(batch))); } else if batch.num_rows() == 0 { // discard empty batches } else { // add to the buffered batches - self.buffer.push(batch.clone()); self.buffered_rows += batch.num_rows(); + self.buffer.push(batch); // check to see if we have enough batches yet if self.buffered_rows >= self.target_batch_size { // combine the batches and return @@ -300,14 +296,14 @@ pub fn concat_batches( batches.len(), row_count ); - let b = arrow::compute::concat_batches(schema, batches)?; - Ok(b) + arrow::compute::concat_batches(schema, batches) } #[cfg(test)] mod tests { use super::*; use crate::{memory::MemoryExec, repartition::RepartitionExec}; + 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 f09b33e3f3b5..bfcff2853538 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -26,11 +26,12 @@ use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::stream::{ObservedStream, RecordBatchReceiverStream}; use super::{DisplayAs, SendableRecordBatchStream, Statistics}; -use crate::{DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; /// Merge execution plan executes partitions in parallel and combines them into a single /// partition. No guarantees are made about the order of the resulting partition. @@ -101,7 +102,10 @@ impl ExecutionPlan for CoalescePartitionsExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() + 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 { diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 649f3a31aa7e..81a59ad7ab3c 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -31,8 +31,10 @@ use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -373,6 +375,38 @@ pub fn batch_byte_size(batch: &RecordBatch) -> usize { batch.get_array_memory_size() } +/// Constructs the mapping between a projection's input and output +pub fn calculate_projection_mapping( + expr: &[(Arc, String)], + input_schema: &Arc, +) -> Result { + // Construct a map from the input expressions to the output expression of the projection: + let mut projection_mapping = vec![]; + for (expr_idx, (expression, name)) in expr.iter().enumerate() { + let target_expr = Arc::new(Column::new(name, expr_idx)) as _; + + let source_expr = expression.clone().transform_down(&|e| match e + .as_any() + .downcast_ref::() + { + Some(col) => { + // Sometimes, expression and its name in the input_schema doesn't match. + // This can cause problems. Hence in here we make sure that expression name + // matches with the name in the inout_schema. + // Conceptually, source_expr and expression should be same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + })?; + + projection_mapping.push((source_expr, target_expr)); + } + Ok(projection_mapping) +} + #[cfg(test)] mod tests { use std::ops::Not; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 59f9928d0e21..ce66d614721c 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -30,7 +30,7 @@ use super::{ use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, - Column, DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, + Column, DisplayFormatType, ExecutionPlan, Partitioning, }; use arrow::compute::filter_record_batch; @@ -42,13 +42,12 @@ use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; -use datafusion_physical_expr::{ - analyze, split_conjunction, AnalysisContext, ExprBoundaries, - OrderingEquivalenceProperties, PhysicalExpr, -}; - 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, +}; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -146,37 +145,29 @@ impl ExecutionPlan for FilterExec { } fn equivalence_properties(&self) -> EquivalenceProperties { + let stats = self.statistics().unwrap(); // Combine the equal predicates with the input equivalence properties - let mut input_properties = self.input.equivalence_properties(); - let (equal_pairs, _ne_pairs) = collect_columns_from_predicate(&self.predicate); - for new_condition in equal_pairs { - input_properties.add_equal_conditions(new_condition) + 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) } - input_properties - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let stats = self - .statistics() - .expect("Ordering equivalences need to handle the error case of statistics"); // 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 Arc) - .collect::>(); - let filter_oeq = self.input.ordering_equivalence_properties(); - filter_oeq.with_constants(constants) + .map(|column| Arc::new(column) as _); + result.add_constants(constants) } fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { - Ok(Arc::new(FilterExec::try_new( - self.predicate.clone(), - children[0].clone(), - )?)) + FilterExec::try_new(self.predicate.clone(), children.swap_remove(0)) + .map(|e| Arc::new(e) as _) } fn execute( @@ -355,17 +346,16 @@ impl RecordBatchStream for FilterExecStream { /// Return the equals Column-Pairs and Non-equals Column-Pairs fn collect_columns_from_predicate(predicate: &Arc) -> EqualAndNonEqual { - let mut eq_predicate_columns: Vec<(&Column, &Column)> = Vec::new(); - let mut ne_predicate_columns: Vec<(&Column, &Column)> = Vec::new(); + let mut eq_predicate_columns = Vec::<(&Column, &Column)>::new(); + let mut ne_predicate_columns = Vec::<(&Column, &Column)>::new(); let predicates = split_conjunction(predicate); predicates.into_iter().for_each(|p| { if let Some(binary) = p.as_any().downcast_ref::() { - let left = binary.left(); - let right = binary.right(); - if left.as_any().is::() && right.as_any().is::() { - let left_column = left.as_any().downcast_ref::().unwrap(); - let right_column = right.as_any().downcast_ref::().unwrap(); + if let (Some(left_column), Some(right_column)) = ( + binary.left().as_any().downcast_ref::(), + binary.right().as_any().downcast_ref::(), + ) { match binary.op() { Operator::Eq => { eq_predicate_columns.push((left_column, right_column)) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index d8c8064e2ac1..102f0c42e90c 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -27,23 +27,22 @@ use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; use crate::{ coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, - ColumnStatistics, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalSortExpr, 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, DataFusionError, Result, ScalarValue}; +use datafusion_common::{plan_err, DataFusionError, 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 datafusion_physical_expr::equivalence::cross_join_equivalence_properties; -use futures::{ready, StreamExt}; -use futures::{Stream, TryStreamExt}; +use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the left side type JoinLeftData = (RecordBatch, MemoryReservation); @@ -106,12 +105,11 @@ async fn load_left_input( reservation: MemoryReservation, ) -> Result { // merge all left parts into a single stream - let merge = { - if left.output_partitioning().partition_count() != 1 { - Arc::new(CoalescePartitionsExec::new(left.clone())) - } else { - left.clone() - } + let left_schema = left.schema(); + let merge = if left.output_partitioning().partition_count() != 1 { + Arc::new(CoalescePartitionsExec::new(left)) + } else { + left }; let stream = merge.execute(0, context)?; @@ -136,7 +134,7 @@ async fn load_left_input( ) .await?; - let merged_batch = concat_batches(&left.schema(), &batches, num_rows)?; + let merged_batch = concat_batches(&left_schema, &batches, num_rows)?; Ok((merged_batch, reservation)) } @@ -217,12 +215,14 @@ impl ExecutionPlan for CrossJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - cross_join_equivalence_properties( + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, + &JoinType::Full, self.schema(), + &[false, false], + None, + &[], ) } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index dc0e81a6f36e..0e2d552b543b 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -42,8 +42,8 @@ use crate::{ BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, - PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use super::{ @@ -55,6 +55,7 @@ use arrow::array::{ Array, ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray, UInt32Array, UInt32BufferBuilder, UInt64Array, UInt64BufferBuilder, }; +use arrow::compute::kernels::cmp::{eq, not_distinct}; use arrow::compute::{and, take, FilterBuilder}; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; @@ -66,13 +67,10 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::EquivalenceProperties; use ahash::RandomState; -use arrow::compute::kernels::cmp::{eq, not_distinct}; -use datafusion_physical_expr::equivalence::{ - combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, -}; use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); @@ -482,26 +480,14 @@ impl ExecutionPlan for HashJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, - self.on(), - self.schema(), - ) - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - combine_join_ordering_equivalence_properties( &self.join_type, - &self.left.ordering_equivalence_properties(), - &self.right.ordering_equivalence_properties(), self.schema(), &self.maintains_input_order(), Some(Self::probe_side()), - self.equivalence_properties(), + self.on(), ) } @@ -533,6 +519,7 @@ impl ExecutionPlan for HashJoinExec { let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); let left_partitions = self.left.output_partitioning().partition_count(); let right_partitions = self.right.output_partitioning().partition_count(); + if self.mode == PartitionMode::Partitioned && left_partitions != right_partitions { return internal_err!( @@ -635,16 +622,10 @@ async fn collect_left_input( let (left_input, left_input_partition) = if let Some(partition) = partition { (left, partition) + } else if left.output_partitioning().partition_count() != 1 { + (Arc::new(CoalescePartitionsExec::new(left)) as _, 0) } else { - let merge = { - if left.output_partitioning().partition_count() != 1 { - Arc::new(CoalescePartitionsExec::new(left)) - } else { - left - } - }; - - (merge, 0) + (left, 0) }; // Depending on partition argument load single partition or whole left side in memory @@ -1175,24 +1156,22 @@ impl Stream for HashJoinStream { mod tests { use std::sync::Arc; - use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder}; - use arrow::datatypes::{DataType, Field, Schema}; - - use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::Literal; - use hashbrown::raw::RawTable; - + use super::*; use crate::{ common, expressions::Column, hash_utils::create_hashes, joins::hash_join::build_equal_condition_join_indices, memory::MemoryExec, repartition::RepartitionExec, test::build_table_i32, test::exec::MockExec, }; + + use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; - use datafusion_physical_expr::expressions::BinaryExpr; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; - use super::*; + use hashbrown::raw::RawTable; fn build_table( a: (&str, &Vec), diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index a113066e39d1..5a77ed6e2907 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -50,7 +50,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::JoinType; use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; -use datafusion_physical_expr::equivalence::combine_join_equivalence_properties; +use datafusion_physical_expr::equivalence::join_equivalence_properties; use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the inner table side @@ -192,14 +192,15 @@ impl ExecutionPlan for NestedLoopJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, - &[], // empty join keys + &self.join_type, self.schema(), + &self.maintains_input_order(), + None, + // No on columns in nested loop join + &[], ) } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 759149a64d9f..f6fdc6d77c0c 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -37,9 +37,8 @@ use crate::joins::utils::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ - metrics, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::*; @@ -52,11 +51,9 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; -use datafusion_physical_expr::equivalence::{ - combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, -}; use futures::{Stream, StreamExt}; /// join execution plan executes partitions in parallel and combines them into a set of @@ -285,26 +282,14 @@ impl ExecutionPlan for SortMergeJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, - self.on(), - self.schema(), - ) - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - combine_join_ordering_equivalence_properties( &self.join_type, - &self.left.ordering_equivalence_properties(), - &self.right.ordering_equivalence_properties(), self.schema(), &self.maintains_input_order(), Some(Self::probe_side(&self.join_type)), - self.equivalence_properties(), + self.on(), ) } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 00d43aead434..3617893a1c61 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -63,10 +63,10 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::ExprIntervalGraph; use ahash::RandomState; -use datafusion_physical_expr::equivalence::combine_join_equivalence_properties; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; use hashbrown::HashSet; @@ -430,14 +430,15 @@ impl ExecutionPlan for SymmetricHashJoinExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, + join_equivalence_properties( self.left.equivalence_properties(), self.right.equivalence_properties(), - left_columns_len, - self.on(), + &self.join_type, self.schema(), + &self.maintains_input_order(), + // Has alternating probe side + None, + self.on(), ) } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index cf150ddf575f..c91dc92fbc7a 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -40,12 +40,12 @@ use datafusion_common::{ plan_datafusion_err, plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; +use datafusion_physical_expr::equivalence::add_offset_to_expr; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ - add_offset_to_expr, add_offset_to_lex_ordering, LexOrdering, LexOrderingRef, - PhysicalExpr, PhysicalSortExpr, + LexOrdering, LexOrderingRef, PhysicalExpr, PhysicalSortExpr, }; use futures::future::{BoxFuture, Shared}; @@ -91,8 +91,8 @@ fn check_join_set_is_valid( if !left_missing.is_empty() | !right_missing.is_empty() { return plan_err!( - "The left or right side of the join does not have all columns on \"on\": \nMissing on the left: {left_missing:?}\nMissing on the right: {right_missing:?}" - ); + "The left or right side of the join does not have all columns on \"on\": \nMissing on the left: {left_missing:?}\nMissing on the right: {right_missing:?}" + ); }; Ok(()) @@ -133,7 +133,7 @@ pub fn adjust_right_output_partitioning( let new_exprs = exprs .into_iter() .map(|expr| add_offset_to_expr(expr, left_columns_len)) - .collect::>(); + .collect(); Partitioning::Hash(new_exprs, size) } } @@ -169,23 +169,22 @@ pub fn calculate_join_output_ordering( maintains_input_order: &[bool], probe_side: Option, ) -> Option { - // All joins have 2 children: - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; let mut right_ordering = match join_type { // In the case below, right ordering should be offseted with the left // side length, since we append the right table to the left table. JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - add_offset_to_lex_ordering(right_ordering, left_columns_len) + right_ordering + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: add_offset_to_expr(sort_expr.expr.clone(), left_columns_len), + options: sort_expr.options, + }) + .collect() } _ => right_ordering.to_vec(), }; - let output_ordering = match (left_maintains, right_maintains) { - (true, true) => { - unreachable!("Cannot maintain ordering of both sides"); - } - (true, false) => { + let output_ordering = match maintains_input_order { + [true, false] => { // Special case, we can prefix ordering of right side with the ordering of left side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { replace_on_columns_of_right_ordering( @@ -198,7 +197,7 @@ pub fn calculate_join_output_ordering( left_ordering.to_vec() } } - (false, true) => { + [false, true] => { // Special case, we can prefix ordering of left side with the ordering of right side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { replace_on_columns_of_right_ordering( @@ -212,7 +211,9 @@ pub fn calculate_join_output_ordering( } } // Doesn't maintain ordering, output ordering is None. - (false, false) => return None, + [false, false] => return None, + [true, true] => unreachable!("Cannot maintain ordering of both sides"), + _ => unreachable!("Join operators can not have more than two children"), }; (!output_ordering.is_empty()).then_some(output_ordering) } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index ed795a1cb333..694a6d3e5c3e 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -33,7 +33,6 @@ use datafusion_common::tree_node::Transformed; use datafusion_common::utils::DataPtr; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::equivalence::OrderingEquivalenceProperties; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ EquivalenceProperties, PhysicalSortExpr, PhysicalSortRequirement, @@ -83,8 +82,7 @@ pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ - expressions, functions, ordering_equivalence_properties_helper, udf, AggregateExpr, - Distribution, Partitioning, PhysicalExpr, + expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, }; // Backwards compatibility @@ -205,16 +203,11 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { .collect() } - /// Get the EquivalenceProperties within the plan + /// Get the [`EquivalenceProperties`] within the plan fn equivalence_properties(&self) -> EquivalenceProperties { EquivalenceProperties::new(self.schema()) } - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - OrderingEquivalenceProperties::new(self.schema()) - } - /// Get a list of `ExecutionPlan` that provide input for 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 diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 9fe1eb0763ba..945dad16b794 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -35,7 +35,6 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -138,10 +137,6 @@ impl ExecutionPlan for GlobalLimitExec { self.input.equivalence_properties() } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() - } - fn with_new_children( self: Arc, children: Vec>, @@ -327,10 +322,6 @@ impl ExecutionPlan for LocalLimitExec { self.input.equivalence_properties() } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() - } - fn unbounded_output(&self, _children: &[bool]) -> Result { Ok(false) } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index b53500579266..5f1660a225b9 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -27,13 +27,12 @@ use super::{ common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::ordering_equivalence_properties_helper; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, project_schema, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use futures::Stream; @@ -122,8 +121,8 @@ impl ExecutionPlan for MemoryExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - ordering_equivalence_properties_helper(self.schema(), &self.sort_information) + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings(self.schema(), &self.sort_information) } fn with_new_children( @@ -179,7 +178,7 @@ impl MemoryExec { } /// A memory table can be ordered by multiple expressions simultaneously. - /// `OrderingEquivalenceProperties` keeps track of expressions that describe the + /// [`EquivalenceProperties`] keeps track of expressions that describe the /// global ordering of the schema. These columns are not necessarily same; e.g. /// ```text /// ┌-------┐ @@ -192,10 +191,8 @@ impl MemoryExec { /// └---┴---┘ /// ``` /// where both `a ASC` and `b DESC` can describe the table ordering. With - /// `OrderingEquivalenceProperties`, we can keep track of these equivalences - /// and treat `a ASC` and `b DESC` as the same ordering requirement - /// by outputting the `a ASC` from output_ordering API - /// and add `b DESC` into `OrderingEquivalenceProperties` + /// [`EquivalenceProperties`], we can keep track of these equivalences + /// and treat `a ASC` and `b DESC` as the same ordering requirement. pub fn with_sort_information(mut self, sort_information: Vec) -> Self { self.sort_information = sort_information; self @@ -303,11 +300,8 @@ mod tests { .with_sort_information(sort_information); assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); - let order_eq = mem_exec.ordering_equivalence_properties(); - assert!(order_eq - .oeq_class() - .map(|class| class.contains(&expected_order_eq)) - .unwrap_or(false)); + let eq_properties = mem_exec.equivalence_properties(); + assert!(eq_properties.oeq_class().contains(&expected_order_eq)); Ok(()) } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index a374154c995c..d4242f20a8a3 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,9 +29,9 @@ use std::task::{Context, Poll}; use super::expressions::{Column, PhysicalSortExpr}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; +use crate::common::calculate_projection_mapping; use crate::{ - ColumnStatistics, DisplayFormatType, EquivalenceProperties, ExecutionPlan, - Partitioning, PhysicalExpr, + ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; use arrow::datatypes::{Field, Schema, SchemaRef}; @@ -40,11 +40,7 @@ use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; -use datafusion_physical_expr::utils::find_orderings_of_exprs; -use datafusion_physical_expr::{ - normalize_out_expr_with_columns_map, project_equivalence_properties, - project_ordering_equivalence_properties, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::EquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -60,15 +56,12 @@ pub struct ProjectionExec { input: Arc, /// The output ordering output_ordering: Option>, - /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr - /// The key is the column from the input schema and the values are the columns from the output schema - columns_map: HashMap>, + /// The mapping used to normalize expressions like Partitioning and + /// PhysicalSortExpr. The key is the expression from the input schema + /// and the value is the expression from the output schema. + projection_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Expressions' normalized orderings (as given by the output ordering API - /// and normalized with respect to equivalence classes of input plan). The - /// projected expressions are mapped by their indices to this vector. - orderings: Vec>, } impl ProjectionExec { @@ -100,63 +93,20 @@ impl ProjectionExec { input_schema.metadata().clone(), )); - // construct a map from the input columns to the output columns of the Projection - let mut columns_map: HashMap> = HashMap::new(); - for (expr_idx, (expression, name)) in expr.iter().enumerate() { - if let Some(column) = expression.as_any().downcast_ref::() { - // For some executors, logical and physical plan schema fields - // are not the same. The information in a `Column` comes from - // the logical plan schema. Therefore, to produce correct results - // we use the field in the input schema with the same index. This - // corresponds to the physical plan `Column`. - let idx = column.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = Column::new(matching_input_field.name(), idx); - let entry = columns_map.entry(matching_input_column).or_default(); - entry.push(Column::new(name, expr_idx)); - }; - } - - // Output Ordering need to respect the alias - let child_output_ordering = input.output_ordering(); - let output_ordering = match child_output_ordering { - Some(sort_exprs) => { - let normalized_exprs = sort_exprs - .iter() - .map(|sort_expr| { - let expr = normalize_out_expr_with_columns_map( - sort_expr.expr.clone(), - &columns_map, - ); - PhysicalSortExpr { - expr, - options: sort_expr.options, - } - }) - .collect::>(); - Some(normalized_exprs) - } - None => None, - }; - - let orderings = find_orderings_of_exprs( - &expr, - input.output_ordering(), - input.equivalence_properties(), - input.ordering_equivalence_properties(), - )?; + // construct a map from the input expressions to the output expression of the Projection + let projection_mapping = calculate_projection_mapping(&expr, &input_schema)?; - let output_ordering = - validate_output_ordering(output_ordering, &orderings, &expr); + let input_eqs = input.equivalence_properties(); + let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); + let output_ordering = project_eqs.oeq_class().output_ordering(); Ok(Self { expr, schema, input, output_ordering, - columns_map, + projection_mapping, metrics: ExecutionPlanMetricsSet::new(), - orderings, }) } @@ -224,11 +174,18 @@ impl ExecutionPlan for ProjectionExec { 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| normalize_out_expr_with_columns_map(expr, &self.columns_map)) - .collect::>(); + .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 @@ -245,58 +202,17 @@ impl ExecutionPlan for ProjectionExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(self.schema()); - project_equivalence_properties( - self.input.equivalence_properties(), - &self.columns_map, - &mut new_properties, - ); - new_properties - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let mut new_properties = OrderingEquivalenceProperties::new(self.schema()); - if self.output_ordering.is_none() { - // If there is no output ordering, return an "empty" equivalence set: - return new_properties; - } - - let input_oeq = self.input().ordering_equivalence_properties(); - - project_ordering_equivalence_properties( - input_oeq, - &self.columns_map, - &mut new_properties, - ); - - if let Some(leading_ordering) = self - .output_ordering - .as_ref() - .map(|output_ordering| &output_ordering[0]) - { - for order in self.orderings.iter().flatten() { - if !order.eq(leading_ordering) - && !new_properties.satisfies_leading_ordering(order) - { - new_properties.add_equal_conditions(( - &vec![leading_ordering.clone()], - &vec![order.clone()], - )); - } - } - } - - new_properties + self.input + .equivalence_properties() + .project(&self.projection_mapping, self.schema()) } fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { - Ok(Arc::new(ProjectionExec::try_new( - self.expr.clone(), - children[0].clone(), - )?)) + ProjectionExec::try_new(self.expr.clone(), children.swap_remove(0)) + .map(|p| Arc::new(p) as _) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -336,40 +252,6 @@ impl ExecutionPlan for ProjectionExec { } } -/// This function takes the current `output_ordering`, the `orderings` based on projected expressions, -/// and the `expr` representing the projected expressions themselves. It aims to ensure that the output -/// ordering is valid and correctly corresponds to the projected columns. -/// -/// If the leading expression in the `output_ordering` is an [`UnKnownColumn`], it indicates that the column -/// referenced in the ordering is not found among the projected expressions. In such cases, this function -/// attempts to create a new output ordering by referring to valid columns from the leftmost side of the -/// expressions that have an ordering specified. -fn validate_output_ordering( - output_ordering: Option>, - orderings: &[Option], - expr: &[(Arc, String)], -) -> Option> { - output_ordering.and_then(|ordering| { - // If the leading expression is invalid column, change output - // ordering of the projection so that it refers to valid columns if - // possible. - if ordering[0].expr.as_any().is::() { - for (idx, order) in orderings.iter().enumerate() { - if let Some(sort_expr) = order { - let (_, col_name) = &expr[idx]; - return Some(vec![PhysicalSortExpr { - expr: Arc::new(Column::new(col_name, idx)), - options: sort_expr.options, - }]); - } - } - None - } else { - Some(ordering) - } - }) -} - /// If e is a direct column reference, returns the field level /// metadata for that field, if any. Otherwise returns None fn get_field_metadata( @@ -486,6 +368,7 @@ mod tests { use crate::common::collect; use crate::expressions; use crate::test; + use arrow_schema::DataType; use datafusion_common::ScalarValue; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 82801bcdfdb3..66f7037e5c2d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -29,9 +29,7 @@ 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, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics, -}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; use self::distributor_channels::{DistributionReceiver, DistributionSender}; @@ -46,7 +44,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalExpr}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use futures::stream::Stream; use futures::{FutureExt, StreamExt}; @@ -427,12 +425,11 @@ impl ExecutionPlan for RepartitionExec { fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { let repartition = - RepartitionExec::try_new(children[0].clone(), self.partitioning.clone())? - .with_preserve_order(self.preserve_order); - Ok(Arc::new(repartition)) + RepartitionExec::try_new(children.swap_remove(0), self.partitioning.clone()); + repartition.map(|r| Arc::new(r.with_preserve_order(self.preserve_order)) as _) } /// Specifies whether this plan generates an infinite stream of records. @@ -468,11 +465,15 @@ impl ExecutionPlan for RepartitionExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + 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(); + } + if self.preserve_order { + result = result.with_reorder(self.sort_exprs().unwrap_or_default().to_vec()) + } + result } fn execute( diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 08fa2c25d792..2d8237011fff 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -835,7 +835,10 @@ impl ExecutionPlan for SortExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() + // Reset the ordering equivalence class with the new ordering: + self.input + .equivalence_properties() + .with_reorder(self.expr.to_vec()) } fn with_new_children( diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index ee044d9172fb..65cd8e41480e 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -33,9 +33,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; use log::{debug, trace}; @@ -176,11 +174,8 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + let output_oeq = self.input.equivalence_properties(); + output_oeq.with_reorder(self.expr.to_vec()) } fn children(&self) -> Vec> { @@ -271,6 +266,8 @@ impl ExecutionPlan for SortPreservingMergeExec { #[cfg(test)] mod tests { + use std::iter::FromIterator; + use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::expressions::col; @@ -281,8 +278,8 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; - use arrow::array::ArrayRef; - use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; + + use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampNanosecondArray}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; @@ -290,7 +287,6 @@ mod tests { use datafusion_execution::config::SessionConfig; use futures::{FutureExt, StreamExt}; - use std::iter::FromIterator; #[tokio::test] async fn test_merge_interleave() { diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 27f03b727c29..1923a5f3abad 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Execution plan for streaming [`PartitionStream`] +//! Generic plans for deferred execution: [`StreamingTableExec`] and [`PartitionStream`] use std::any::Any; use std::sync::Arc; @@ -28,13 +28,17 @@ use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use async_trait::async_trait; use futures::stream::StreamExt; use log::debug; /// A partition that can be converted into a [`SendableRecordBatchStream`] +/// +/// Combined with [`StreamingTableExec`], you can use this trait to implement +/// [`ExecutionPlan`] for a custom source with less boiler plate than +/// implementing `ExecutionPlan` directly for many use cases. pub trait PartitionStream: Send + Sync { /// Returns the schema of this partition fn schema(&self) -> &SchemaRef; @@ -43,7 +47,10 @@ pub trait PartitionStream: Send + Sync { fn execute(&self, ctx: Arc) -> SendableRecordBatchStream; } -/// An [`ExecutionPlan`] for [`PartitionStream`] +/// An [`ExecutionPlan`] for one or more [`PartitionStream`]s. +/// +/// If your source can be represented as one or more [`PartitionStream`]s, you can +/// use this struct to implement [`ExecutionPlan`]. pub struct StreamingTableExec { partitions: Vec>, projection: Option>, @@ -156,6 +163,14 @@ impl ExecutionPlan for StreamingTableExec { self.projected_output_ordering.as_deref() } + fn equivalence_properties(&self) -> EquivalenceProperties { + let mut result = EquivalenceProperties::new(self.schema()); + if let Some(ordering) = &self.projected_output_ordering { + result.add_new_orderings([ordering.clone()]) + } + result + } + fn children(&self) -> Vec> { vec![] } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 2727a6e161be..9700605ce406 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -40,6 +40,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; use futures::Stream; use itertools::Itertools; @@ -222,6 +223,46 @@ 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>, @@ -596,10 +637,41 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { mod tests { use super::*; use crate::collect; + use crate::memory::MemoryExec; use crate::test; use arrow::record_batch::RecordBatch; + use arrow_schema::{DataType, SortOptions}; use datafusion_common::ScalarValue; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::PhysicalExpr; + + // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let g = Field::new("g", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g])); + + Ok(schema) + } + + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs( + in_data: &[(&Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect::>() + } #[tokio::test] async fn test_union_partitions() -> Result<()> { @@ -712,4 +784,105 @@ mod tests { assert_eq!(result, expected); } + + #[tokio::test] + async fn test_union_equivalence_properties() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; + let col_f = &col("f", &schema)?; + let options = SortOptions::default(); + let test_cases = vec![ + //-----------TEST CASE 1----------// + ( + // First child orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + // Second child orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + // Union output orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + ), + //-----------TEST CASE 2----------// + ( + // First child orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + // d ASC + vec![(col_d, options)], + ], + // Second child orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [e ASC] + vec![(col_e, options)], + ], + // Union output orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, options), (col_b, options)], + ], + ), + ]; + + for ( + test_idx, + (first_child_orderings, second_child_orderings, union_orderings), + ) in test_cases.iter().enumerate() + { + let first_orderings = first_child_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let second_orderings = second_child_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let union_expected_orderings = union_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let child1 = Arc::new( + MemoryExec::try_new(&[], schema.clone(), None)? + .with_sort_information(first_orderings), + ); + let child2 = Arc::new( + MemoryExec::try_new(&[], schema.clone(), None)? + .with_sort_information(second_orderings), + ); + + let union = UnionExec::new(vec![child1, child2]); + let union_eq_properties = union.equivalence_properties(); + let union_actual_orderings = union_eq_properties.oeq_class(); + let err_msg = format!( + "Error in test id: {:?}, test case: {:?}", + test_idx, test_cases[test_idx] + ); + assert_eq!( + union_actual_orderings.len(), + union_expected_orderings.len(), + "{}", + err_msg + ); + for expected in &union_expected_orderings { + assert!(union_actual_orderings.contains(expected), "{}", err_msg); + } + } + Ok(()) + } } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 30f109953cbb..c9f3fb76c2e5 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -23,9 +23,8 @@ use std::{any::Any, sync::Arc}; use super::DisplayAs; use crate::{ - expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, + expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalExpr, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, }; use arrow::array::{ @@ -136,10 +135,6 @@ impl ExecutionPlan for UnnestExec { None } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } - fn execute( &self, partition: usize, 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 f6ffe2e26795..fb679b013863 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -30,8 +30,8 @@ use std::task::{Context, Poll}; use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ - calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, - PartitionSearchMode, + calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, + window_equivalence_properties, PartitionSearchMode, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -58,8 +58,7 @@ use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; use datafusion_physical_expr::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalExpr, - PhysicalSortRequirement, + EquivalenceProperties, PhysicalExpr, PhysicalSortRequirement, }; use ahash::RandomState; @@ -150,9 +149,12 @@ impl BoundedWindowAggExec { // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points pub fn partition_by_sort_keys(&self) -> Result> { - // Partition by sort keys indices are stored in self.ordered_partition_by_indices. - let sort_keys = self.input.output_ordering().unwrap_or(&[]); - get_at_indices(sort_keys, &self.ordered_partition_by_indices) + let partition_by = self.window_expr()[0].partition_by(); + get_partition_by_sort_exprs( + &self.input, + partition_by, + &self.ordered_partition_by_indices, + ) } /// Initializes the appropriate [`PartitionSearcher`] implementation from @@ -264,13 +266,9 @@ impl ExecutionPlan for BoundedWindowAggExec { } } + /// Get the [`EquivalenceProperties`] within the plan fn equivalence_properties(&self) -> EquivalenceProperties { - self.input().equivalence_properties() - } - - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) + window_equivalence_properties(&self.schema, &self.input, &self.window_expr) } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index aff936499a5e..26dddc4ddde4 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -32,24 +32,18 @@ use crate::{ use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; -use datafusion_common::utils::{ - find_indices, get_at_indices, is_sorted, longest_consecutive_prefix, - merge_and_order_indices, set_difference, -}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::{ window_function::{BuiltInWindowFunction, WindowFunction}, PartitionEvaluator, WindowFrame, WindowUDF, }; +use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ - equivalence::OrderingEquivalenceBuilder, - utils::{convert_to_expr, get_indices_of_matching_exprs}, + reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, - AggregateExpr, OrderingEquivalenceProperties, PhysicalSortRequirement, + AggregateExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; -use itertools::{izip, Itertools}; - mod bounded_window_agg_exec; mod window_agg_exec; @@ -61,13 +55,19 @@ pub use datafusion_physical_expr::window::{ }; #[derive(Debug, Clone, PartialEq)] -/// Specifies partition column properties in terms of input ordering +/// Specifies partition expression properties in terms of existing ordering(s). +/// As an example if existing ordering is [a ASC, b ASC, c ASC], +/// `PARTITION BY b` will have `PartitionSearchMode::Linear`. +/// `PARTITION BY a, c` and `PARTITION BY c, a` will have `PartitionSearchMode::PartiallySorted(0)`, `PartitionSearchMode::PartiallySorted(1)` +/// respectively (subset `a` defines an ordered section. Indices points to index of `a` among partition by expressions). +/// `PARTITION BY a, b` and `PARTITION BY b, a` will have `PartitionSearchMode::Sorted` mode. pub enum PartitionSearchMode { - /// None of the columns among the partition columns is ordered. + /// None of the partition expressions is ordered. Linear, - /// Some columns of the partition columns are ordered but not all + /// A non-empty subset of the the partition expressions are ordered. + /// Indices stored constructs ordered subset, that is satisfied by existing ordering(s). PartiallySorted(Vec), - /// All Partition columns are ordered (Also empty case) + /// All Partition expressions are ordered (Also empty case) Sorted, } @@ -331,45 +331,51 @@ pub(crate) fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { - let input_ordering = input.output_ordering().unwrap_or(&[]); - let input_ordering_exprs = convert_to_expr(input_ordering); - let equal_properties = || input.equivalence_properties(); - let input_places = get_indices_of_matching_exprs( - &input_ordering_exprs, - partition_by_exprs, - equal_properties, - ); - let mut partition_places = get_indices_of_matching_exprs( - partition_by_exprs, - &input_ordering_exprs, - equal_properties, - ); - partition_places.sort(); - let first_n = longest_consecutive_prefix(partition_places); - input_places[0..first_n].to_vec() + let (_, indices) = input + .equivalence_properties() + .find_longest_permutation(partition_by_exprs); + indices } -pub(crate) fn window_ordering_equivalence( +pub(crate) fn get_partition_by_sort_exprs( + input: &Arc, + partition_by_exprs: &[Arc], + ordered_partition_by_indices: &[usize], +) -> Result { + let ordered_partition_exprs = ordered_partition_by_indices + .iter() + .map(|idx| partition_by_exprs[*idx].clone()) + .collect::>(); + // Make sure ordered section doesn't move over the partition by expression + assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); + let (ordering, _) = input + .equivalence_properties() + .find_longest_permutation(&ordered_partition_exprs); + if ordering.len() == ordered_partition_exprs.len() { + Ok(ordering) + } else { + exec_err!("Expects PARTITION BY expression to be ordered") + } +} + +pub(crate) fn window_equivalence_properties( schema: &SchemaRef, input: &Arc, window_expr: &[Arc], -) -> OrderingEquivalenceProperties { +) -> EquivalenceProperties { // We need to update the schema, so we can not directly use - // `input.ordering_equivalence_properties()`. - let mut builder = OrderingEquivalenceBuilder::new(schema.clone()) - .with_equivalences(input.equivalence_properties()) - .with_existing_ordering(input.output_ordering().map(|elem| elem.to_vec())) - .extend(input.ordering_equivalence_properties()); + // `input.equivalence_properties()`. + let mut window_eq_properties = + EquivalenceProperties::new(schema.clone()).extend(input.equivalence_properties()); for expr in window_expr { if let Some(builtin_window_expr) = expr.as_any().downcast_ref::() { - builtin_window_expr - .add_equal_orderings(&mut builder, || input.equivalence_properties()); + builtin_window_expr.add_equal_orderings(&mut window_eq_properties); } } - builder.build() + window_eq_properties } /// Constructs the best-fitting windowing operator (a `WindowAggExec` or a @@ -396,7 +402,7 @@ pub fn get_best_fitting_window( let orderby_keys = window_exprs[0].order_by(); let (should_reverse, partition_search_mode) = if let Some((should_reverse, partition_search_mode)) = - can_skip_sort(partitionby_exprs, orderby_keys, input)? + get_window_mode(partitionby_exprs, orderby_keys, input)? { (should_reverse, partition_search_mode) } else { @@ -459,149 +465,41 @@ pub fn get_best_fitting_window( /// The `bool` field in the return value represents whether we should reverse window /// operator to remove `SortExec` before it. The `PartitionSearchMode` field represents /// the mode this window operator should work in to accomodate the existing ordering. -fn can_skip_sort( +pub fn get_window_mode( partitionby_exprs: &[Arc], orderby_keys: &[PhysicalSortExpr], input: &Arc, ) -> Result> { - let physical_ordering = if let Some(physical_ordering) = input.output_ordering() { - physical_ordering - } else { - // If there is no physical ordering, there is no way to remove a - // sort, so immediately return. - return Ok(None); - }; - let orderby_exprs = convert_to_expr(orderby_keys); - let physical_ordering_exprs = convert_to_expr(physical_ordering); - let equal_properties = || input.equivalence_properties(); - // Get the indices of the ORDER BY expressions among input ordering expressions: - let ob_indices = get_indices_of_matching_exprs( - &orderby_exprs, - &physical_ordering_exprs, - equal_properties, - ); - if ob_indices.len() != orderby_exprs.len() { - // If all order by expressions are not in the input ordering, - // there is no way to remove a sort -- immediately return: - return Ok(None); - } - // Get the indices of the PARTITION BY expressions among input ordering expressions: - let pb_indices = get_indices_of_matching_exprs( - partitionby_exprs, - &physical_ordering_exprs, - equal_properties, - ); - let ordered_merged_indices = merge_and_order_indices(&pb_indices, &ob_indices); - // Get the indices of the ORDER BY columns that don't appear in the - // PARTITION BY clause; i.e. calculate (ORDER BY columns) ∖ (PARTITION - // BY columns) where `∖` represents set difference. - let unique_ob_indices = set_difference(&ob_indices, &pb_indices); - if !is_sorted(&unique_ob_indices) { - // ORDER BY indices should be ascending ordered - return Ok(None); - } - let first_n = longest_consecutive_prefix(ordered_merged_indices); - let furthest_ob_index = *unique_ob_indices.last().unwrap_or(&0); - // Cannot skip sort if last order by index is not within consecutive prefix. - // For instance, if input is ordered by a, b, c, d for the expression - // `PARTITION BY a, ORDER BY b, d`, then `first_n` would be 2 (meaning a, b defines a - // prefix for input ordering). However, `furthest_ob_index` would be 3 as column d - // occurs at the 3rd index of the existing ordering. Hence, existing ordering would - // not be sufficient to run the current operator. - // However, for expression `PARTITION BY a, ORDER BY b, c, d`, `first_n` would be 4 (meaning - // a, b, c, d defines a prefix for input ordering). Similarly, `furthest_ob_index` would be - // 3 as column d occurs at the 3rd index of the existing ordering. Therefore, the existing - // ordering would be sufficient to run the current operator. - if first_n <= furthest_ob_index { - return Ok(None); - } - let input_orderby_columns = get_at_indices(physical_ordering, &unique_ob_indices)?; - let expected_orderby_columns = - get_at_indices(orderby_keys, find_indices(&ob_indices, &unique_ob_indices)?)?; - let should_reverse = if let Some(should_reverse) = check_alignments( - &input.schema(), - &input_orderby_columns, - &expected_orderby_columns, - )? { - should_reverse - } else { - // If ordering directions are not aligned, we cannot calculate the - // result without changing existing ordering. - return Ok(None); - }; - - let ordered_pb_indices = pb_indices.iter().copied().sorted().collect::>(); - // Determine how many elements in the PARTITION BY columns defines a consecutive range from zero. - let first_n = longest_consecutive_prefix(&ordered_pb_indices); - let mode = if first_n == partitionby_exprs.len() { - // All of the PARTITION BY columns defines a consecutive range from zero. - PartitionSearchMode::Sorted - } else if first_n > 0 { - // All of the PARTITION BY columns defines a consecutive range from zero. - let ordered_range = &ordered_pb_indices[0..first_n]; - let input_pb_exprs = get_at_indices(&physical_ordering_exprs, ordered_range)?; - let partially_ordered_indices = get_indices_of_matching_exprs( - &input_pb_exprs, - partitionby_exprs, - equal_properties, - ); - PartitionSearchMode::PartiallySorted(partially_ordered_indices) - } else { - // None of the PARTITION BY columns defines a consecutive range from zero. - PartitionSearchMode::Linear - }; - - Ok(Some((should_reverse, mode))) -} - -/// Compares all the orderings in `physical_ordering` and `required`, decides -/// whether alignments match. A `None` return value indicates that current -/// column is not aligned. A `Some(bool)` value indicates otherwise, and signals -/// whether we should reverse the window expression in order to avoid sorting. -fn check_alignments( - schema: &SchemaRef, - physical_ordering: &[PhysicalSortExpr], - required: &[PhysicalSortExpr], -) -> Result> { - let result = izip!(physical_ordering, required) - .map(|(lhs, rhs)| check_alignment(schema, lhs, rhs)) - .collect::>>>()?; - Ok(if let Some(res) = result { - if !res.is_empty() { - let first = res[0]; - let all_same = res.into_iter().all(|elem| elem == first); - all_same.then_some(first) - } else { - Some(false) - } - } else { - // Cannot skip some of the requirements in the input. - None - }) -} - -/// Compares `physical_ordering` and `required` ordering, decides whether -/// alignments match. A `None` return value indicates that current column is -/// not aligned. A `Some(bool)` value indicates otherwise, and signals whether -/// we should reverse the window expression in order to avoid sorting. -fn check_alignment( - input_schema: &SchemaRef, - physical_ordering: &PhysicalSortExpr, - required: &PhysicalSortExpr, -) -> Result> { - Ok(if required.expr.eq(&physical_ordering.expr) { - let physical_opts = physical_ordering.options; - let required_opts = required.options; - if required.expr.nullable(input_schema)? { - let reverse = physical_opts == !required_opts; - (reverse || physical_opts == required_opts).then_some(reverse) - } else { - // If the column is not nullable, NULLS FIRST/LAST is not important. - Some(physical_opts.descending != required_opts.descending) + let input_eqs = input.equivalence_properties(); + 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 { + expr: partitionby_exprs[idx].clone(), + options: None, + })); + // Treat partition by exprs as constant. During analysis of requirements are satisfied. + let partition_by_eqs = input_eqs.add_constants(partitionby_exprs.iter().cloned()); + let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); + let reverse_order_by_reqs = + PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(orderby_keys)); + for (should_swap, order_by_reqs) in + [(false, order_by_reqs), (true, reverse_order_by_reqs)] + { + let req = [partition_by_reqs.clone(), order_by_reqs].concat(); + let req = collapse_lex_req(req); + if partition_by_eqs.ordering_satisfy_requirement(&req) { + // Window can be run with existing ordering + let mode = if indices.len() == partitionby_exprs.len() { + PartitionSearchMode::Sorted + } else if indices.is_empty() { + PartitionSearchMode::Linear + } else { + PartitionSearchMode::PartiallySorted(indices) + }; + return Ok(Some((should_swap, mode))); } - } else { - None - }) + } + Ok(None) } #[cfg(test)] @@ -613,7 +511,6 @@ mod tests { use crate::streaming::StreamingTableExec; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::windows::PartitionSearchMode::{Linear, PartiallySorted, Sorted}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, SchemaRef}; @@ -621,6 +518,8 @@ mod tests { use futures::FutureExt; + use PartitionSearchMode::{Linear, PartiallySorted, Sorted}; + fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); @@ -781,15 +680,16 @@ mod tests { } #[tokio::test] - async fn test_is_column_aligned_nullable() -> Result<()> { + async fn test_satisfiy_nullable() -> Result<()> { let schema = create_test_schema()?; let params = vec![ - ((true, true), (false, false), Some(true)), - ((true, true), (false, true), None), - ((true, true), (true, false), None), - ((true, false), (false, true), Some(true)), - ((true, false), (false, false), None), - ((true, false), (true, true), None), + ((true, true), (false, false), false), + ((true, true), (false, true), false), + ((true, true), (true, false), false), + ((true, false), (false, true), false), + ((true, false), (false, false), false), + ((true, false), (true, true), false), + ((true, false), (true, false), true), ]; for ( (physical_desc, physical_nulls_first), @@ -811,7 +711,7 @@ mod tests { nulls_first: req_nulls_first, }, }; - let res = check_alignment(&schema, &physical_ordering, &required_ordering)?; + let res = physical_ordering.satisfy(&required_ordering.into(), &schema); assert_eq!(res, expected); } @@ -819,16 +719,17 @@ mod tests { } #[tokio::test] - async fn test_is_column_aligned_non_nullable() -> Result<()> { + async fn test_satisfy_non_nullable() -> Result<()> { let schema = create_test_schema()?; let params = vec![ - ((true, true), (false, false), Some(true)), - ((true, true), (false, true), Some(true)), - ((true, true), (true, false), Some(false)), - ((true, false), (false, true), Some(true)), - ((true, false), (false, false), Some(true)), - ((true, false), (true, true), Some(false)), + ((true, true), (false, false), false), + ((true, true), (false, true), false), + ((true, true), (true, false), true), + ((true, false), (false, true), false), + ((true, false), (false, false), false), + ((true, false), (true, true), true), + ((true, false), (true, false), true), ]; for ( (physical_desc, physical_nulls_first), @@ -850,7 +751,7 @@ mod tests { nulls_first: req_nulls_first, }, }; - let res = check_alignment(&schema, &physical_ordering, &required_ordering)?; + let res = physical_ordering.satisfy(&required_ordering.into(), &schema); assert_eq!(res, expected); } @@ -858,7 +759,7 @@ mod tests { } #[tokio::test] - async fn test_can_skip_ordering_exhaustive() -> Result<()> { + async fn test_get_window_mode_exhaustive() -> Result<()> { let test_schema = create_test_schema3()?; // Columns a,c are nullable whereas b,d are not nullable. // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST @@ -891,7 +792,7 @@ mod tests { (vec!["a"], vec!["a", "c"], None), (vec!["a"], vec!["a", "b", "c"], Some(Sorted)), (vec!["b"], vec!["a"], Some(Linear)), - (vec!["b"], vec!["b"], None), + (vec!["b"], vec!["b"], Some(Linear)), (vec!["b"], vec!["c"], None), (vec!["b"], vec!["a", "b"], Some(Linear)), (vec!["b"], vec!["b", "c"], None), @@ -899,7 +800,7 @@ mod tests { (vec!["b"], vec!["a", "b", "c"], Some(Linear)), (vec!["c"], vec!["a"], Some(Linear)), (vec!["c"], vec!["b"], None), - (vec!["c"], vec!["c"], None), + (vec!["c"], vec!["c"], Some(Linear)), (vec!["c"], vec!["a", "b"], Some(Linear)), (vec!["c"], vec!["b", "c"], None), (vec!["c"], vec!["a", "c"], Some(Linear)), @@ -912,10 +813,10 @@ mod tests { (vec!["b", "a"], vec!["a", "c"], Some(Sorted)), (vec!["b", "a"], vec!["a", "b", "c"], Some(Sorted)), (vec!["c", "b"], vec!["a"], Some(Linear)), - (vec!["c", "b"], vec!["b"], None), - (vec!["c", "b"], vec!["c"], None), + (vec!["c", "b"], vec!["b"], Some(Linear)), + (vec!["c", "b"], vec!["c"], Some(Linear)), (vec!["c", "b"], vec!["a", "b"], Some(Linear)), - (vec!["c", "b"], vec!["b", "c"], None), + (vec!["c", "b"], vec!["b", "c"], Some(Linear)), (vec!["c", "b"], vec!["a", "c"], Some(Linear)), (vec!["c", "b"], vec!["a", "b", "c"], Some(Linear)), (vec!["c", "a"], vec!["a"], Some(PartiallySorted(vec![1]))), @@ -965,7 +866,7 @@ mod tests { order_by_exprs.push(PhysicalSortExpr { expr, options }); } let res = - can_skip_sort(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?; + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?; // Since reversibility is not important in this test. Convert Option<(bool, PartitionSearchMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( @@ -978,7 +879,7 @@ mod tests { } #[tokio::test] - async fn test_can_skip_ordering() -> Result<()> { + async fn test_get_window_mode() -> Result<()> { let test_schema = create_test_schema3()?; // Columns a,c are nullable whereas b,d are not nullable. // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST @@ -1129,7 +1030,7 @@ mod tests { } assert_eq!( - can_skip_sort(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?, + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?, *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 4873778425d3..6c245f65ba4f 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -26,12 +26,13 @@ use crate::common::transpose; use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ - calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, + calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, + window_equivalence_properties, }; use crate::{ - ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, WindowExpr, + ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, + Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + WindowExpr, }; use arrow::compute::{concat, concat_batches}; @@ -43,10 +44,10 @@ use arrow::{ record_batch::RecordBatch, }; use datafusion_common::stats::Precision; -use datafusion_common::utils::{evaluate_partition_ranges, get_at_indices}; +use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; use futures::stream::Stream; use futures::{ready, StreamExt}; @@ -107,9 +108,12 @@ impl WindowAggExec { // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points pub fn partition_by_sort_keys(&self) -> Result> { - // Partition by sort keys indices are stored in self.ordered_partition_by_indices. - let sort_keys = self.input.output_ordering().unwrap_or(&[]); - get_at_indices(sort_keys, &self.ordered_partition_by_indices) + let partition_by = self.window_expr()[0].partition_by(); + get_partition_by_sort_exprs( + &self.input, + partition_by, + &self.ordered_partition_by_indices, + ) } } @@ -206,13 +210,9 @@ impl ExecutionPlan for WindowAggExec { } } + /// Get the [`EquivalenceProperties`] within the plan fn equivalence_properties(&self) -> EquivalenceProperties { - self.input().equivalence_properties() - } - - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) + window_equivalence_properties(&self.schema, &self.input, &self.window_expr) } fn with_new_children( diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index ef6fee69abbf..cb0b0b7c76a5 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2019,7 +2019,7 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered +------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0]) --------------SortExec: expr=[col0@3 ASC NULLS LAST] ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] @@ -2118,7 +2118,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, SUM(annotate ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@1 as a, b@0 as b, SUM(annotated_data_infinite2.c)@2 as summation1] ---AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -2149,7 +2149,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotate ----TableScan: annotated_data_infinite2 projection=[a, c, d] physical_plan ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] ---AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallyOrdered +--AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1]) ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true query III @@ -2182,7 +2182,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE( ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2208,7 +2208,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2235,7 +2235,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2354,9 +2354,9 @@ SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country # test_ordering_sensitive_aggregation4 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should append requirement to +# If aggregators can work with bounded memory (Sorted or PartiallySorted mode), we should append requirement to # the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. -# This test checks for whether we can satisfy aggregation requirement in FullyOrdered mode. +# This test checks for whether we can satisfy aggregation requirement in Sorted mode. query TT EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 @@ -2373,7 +2373,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2390,9 +2390,9 @@ GRC [80.0, 30.0] 110 TUR [100.0, 75.0] 175 # test_ordering_sensitive_aggregation5 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# If aggregators can work with bounded memory (Sorted or PartiallySorted mode), we should be append requirement to # the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. -# This test checks for whether we can satisfy aggregation requirement in PartiallyOrdered mode. +# This test checks for whether we can satisfy aggregation requirement in PartiallySorted mode. query TT EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 @@ -2409,7 +2409,7 @@ Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC N --------TableScan: sales_global projection=[zip_code, country, amount] physical_plan ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] ---AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered +--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) ----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2426,7 +2426,7 @@ GRC 0 [80.0, 30.0] 110 TUR 1 [100.0, 75.0] 175 # test_ordering_sensitive_aggregation6 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# If aggregators can work with bounded memory (FullySorted or PartiallySorted mode), we should be append requirement to # the existing ordering. When group by expressions contain aggregation requirement, we shouldn't append redundant expression. # Hence in the final plan SortExec should be SortExec: expr=[country@0 DESC] not SortExec: expr=[country@0 ASC NULLS LAST,country@0 DESC] query TT @@ -2445,7 +2445,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2480,7 +2480,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -3646,6 +3646,18 @@ GROUP BY d; 0 4 0 9 +query III rowsort +SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d; +---- +0 95 0 +1 90 4 +2 97 1 +3 99 15 +4 98 9 + query TT EXPLAIN SELECT c FROM multiple_ordered_table @@ -3656,6 +3668,49 @@ Sort: multiple_ordered_table.c ASC NULLS LAST --TableScan: multiple_ordered_table projection=[c] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +statement ok +set datafusion.execution.target_partitions = 1; + +query TT +EXPLAIN SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd +FROM multiple_ordered_table AS l +INNER JOIN ( + SELECT *, ROW_NUMBER() OVER (ORDER BY r.a) as row_n FROM multiple_ordered_table AS r +) +ON l.d = r.d AND + l.a >= r.a - 10 +GROUP BY row_n +ORDER BY row_n +---- +logical_plan +Projection: amount_usd +--Sort: row_n ASC NULLS LAST +----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +--------Projection: l.a, l.d, row_n +----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +------------SubqueryAlias: l +--------------TableScan: multiple_ordered_table projection=[a, d] +------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----------------SubqueryAlias: r +------------------TableScan: multiple_ordered_table projection=[a, d] +physical_plan +ProjectionExec: expr=[amount_usd@0 as amount_usd] +--ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] +----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +--------CoalesceBatchesExec: target_batch_size=2 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +# reset partition number to 8. +statement ok +set datafusion.execution.target_partitions = 8; + # Create an external table with primary key # column c statement ok @@ -3685,11 +3740,11 @@ logical_plan Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] --TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) --SortExec: expr=[c@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true @@ -3726,11 +3781,11 @@ logical_plan Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] --TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) --SortExec: expr=[c@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered +--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index cc90e6431389..c794c4da4310 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3283,6 +3283,127 @@ CoalesceBatchesExec: target_batch_size=2 ------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +query TT +EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 +FROM annotated_data as l +JOIN annotated_data as r +ON l.a = r.a +GROUP BY l.a, l.b, l.c +ORDER BY l.a ASC NULLS FIRST; +---- +logical_plan +Sort: l.a ASC NULLS FIRST +--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +------Inner Join: l.a = r.a +--------SubqueryAlias: l +----------TableScan: annotated_data projection=[a, b, c] +--------SubqueryAlias: r +----------TableScan: annotated_data projection=[a, b] +physical_plan +ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +----CoalesceBatchesExec: target_batch_size=2 +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true + +# create a table where there more than one valid ordering +# that describes table. +statement ok +CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +query TT +EXPLAIN SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd +FROM multiple_ordered_table AS l +INNER JOIN ( + SELECT *, ROW_NUMBER() OVER (ORDER BY r.a) as row_n FROM multiple_ordered_table AS r +) +ON l.d = r.d AND + l.a >= r.a - 10 +GROUP BY row_n +ORDER BY row_n +---- +logical_plan +Projection: amount_usd +--Sort: row_n ASC NULLS LAST +----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +--------Projection: l.a, l.d, row_n +----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +------------SubqueryAlias: l +--------------TableScan: multiple_ordered_table projection=[a, d] +------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----------------SubqueryAlias: r +------------------TableScan: multiple_ordered_table projection=[a, d] +physical_plan +ProjectionExec: expr=[amount_usd@0 as amount_usd] +--ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] +----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +--------CoalesceBatchesExec: target_batch_size=2 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +# run query above in multiple partitions +statement ok +set datafusion.execution.target_partitions = 2; + +# use bounded variants +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + +query TT +EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 +FROM annotated_data as l +JOIN annotated_data as r +ON l.a = r.a +GROUP BY l.a, l.b, l.c +ORDER BY l.a ASC NULLS FIRST; +---- +logical_plan +Sort: l.a ASC NULLS FIRST +--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +------Inner Join: l.a = r.a +--------SubqueryAlias: l +----------TableScan: annotated_data projection=[a, b, c] +--------SubqueryAlias: r +----------TableScan: annotated_data projection=[a, b] +physical_plan +SortPreservingMergeExec: [a@0 ASC] +--SortExec: expr=[a@0 ASC] +----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 +------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +--------------CoalesceBatchesExec: target_batch_size=2 +----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +------------------CoalesceBatchesExec: target_batch_size=2 +--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +------------------CoalesceBatchesExec: target_batch_size=2 +--------------------SortPreservingRepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, sort_exprs=a@0 ASC,b@1 ASC NULLS LAST +----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true + #### # Config teardown #### @@ -3296,5 +3417,9 @@ set datafusion.optimizer.prefer_hash_join = true; statement ok set datafusion.execution.target_partitions = 2; +statement ok +set datafusion.optimizer.prefer_existing_sort = false; + statement ok drop table annotated_data; + diff --git a/datafusion/sqllogictest/test_files/tpch/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/q17.slt.part index 50661b9b10a8..4d4aa4b1395f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -58,21 +58,19 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av --------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice] ----------CoalesceBatchesExec: target_batch_size=8192 ------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([p_partkey@2], 4), input_partitions=4 -------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] +--------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false +----------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX +------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false --------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] ----------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] ------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 213f6daaef3e..c7060433d999 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -357,7 +357,7 @@ Sort: d.b ASC NULLS LAST physical_plan SortPreservingMergeExec: [b@0 ASC NULLS LAST] --ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] -----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=FullyOrdered +----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted ------ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] --------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }], mode=[Sorted] ----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] @@ -1957,7 +1957,7 @@ Sort: aggregate_test_100.c1 ASC NULLS LAST ----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] ------TableScan: aggregate_test_100 projection=[c1] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] --ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] ----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] ------SortExec: expr=[c1@0 ASC NULLS LAST] @@ -2132,15 +2132,12 @@ ProjectionExec: expr=[c9@1 as c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER B ----BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] ------ProjectionExec: expr=[c2@0 as c2, c9@2 as c9, c1_alias@3 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] --------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)) }] -----------SortExec: expr=[c2@0 ASC NULLS LAST,c1_alias@3 ASC NULLS LAST,c9@2 ASC NULLS LAST,c8@1 ASC NULLS LAST] -------------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] ---------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] -----------------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)) }] -------------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] ---------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true - - +----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] +--------------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)) }] +----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] +--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true query IIIII SELECT c9, @@ -2338,10 +2335,11 @@ Limit: skip=0, fetch=5 ----------TableScan: aggregate_test_100 projection=[c9] physical_plan GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +--SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] +----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------SortExec: expr=[c9@0 DESC] +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -3241,7 +3239,7 @@ physical_plan ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum4] --BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] ----CoalesceBatchesExec: target_batch_size=4096 -------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b ASC NULLS LAST,c ASC NULLS LAST +------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST --------ProjectionExec: expr=[a@0 as a, d@3 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ------------CoalesceBatchesExec: target_batch_size=4096 @@ -3379,6 +3377,96 @@ SELECT ORDER BY C3 LIMIT 5 +# Create a source where there is multiple orderings. +statement ok +CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +# All of the window execs in the physical plan should work in the +# sorted mode. +query TT +EXPLAIN SELECT MIN(d) OVER(ORDER BY c ASC) as min1, + MAX(d) OVER(PARTITION BY b, a ORDER BY c ASC) as max1 +FROM multiple_ordered_table +---- +logical_plan +Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 +--WindowAggr: windowExpr=[[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----Projection: multiple_ordered_table.c, multiple_ordered_table.d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +------WindowAggr: windowExpr=[[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--------TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] +--BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true + +query TT +EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c +FROM( + SELECT * + FROM multiple_ordered_table + WHERE d=0) +---- +logical_plan +Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c +--WindowAggr: windowExpr=[[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----Filter: multiple_ordered_table.d = Int32(0) +------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] +physical_plan +ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] +--BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CoalesceBatchesExec: target_batch_size=4096 +------FilterExec: d@1 = 0 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true + +query TT +explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) +FROM multiple_ordered_table; +---- +logical_plan +Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----TableScan: multiple_ordered_table projection=[a, c, d] +physical_plan +ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +query TT +explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) +FROM multiple_ordered_table; +---- +logical_plan +Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true + +query I +SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) +FROM multiple_ordered_table +LIMIT 5; +---- +0 +2 +0 +0 +1 # simple window query query II