From 0444abc2df2eda6ef59ea15825b739ea43abef4e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 25 Jan 2023 21:18:21 -0600 Subject: [PATCH 01/31] Make EnforceSorting global sort aware, fix union bug, parallelize CP+Sort cascades --- datafusion/core/src/execution/context.rs | 7 +- .../physical_optimizer/dist_enforcement.rs | 2 +- .../src/physical_optimizer/repartition.rs | 2 +- .../physical_optimizer/sort_enforcement.rs | 434 +++++++++++++++--- .../core/src/physical_optimizer/test_utils.rs | 7 +- datafusion/core/tests/sql/explain_analyze.rs | 5 - datafusion/core/tests/sql/joins.rs | 32 +- datafusion/core/tests/sql/window.rs | 56 ++- 8 files changed, 456 insertions(+), 89 deletions(-) diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 1df9bf111ed16..a08f7babbc3cd 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1546,8 +1546,11 @@ impl SessionState { // The EnforceSorting rule is for adding essential local sorting to satisfy the required // ordering. Please make sure that the whole plan tree is determined before this rule. // Note that one should always run this rule after running the EnforceDistribution rule - // as the latter may break local sorting requirements. - Arc::new(EnforceSorting::new()), + // as the latter may break local sorting requirements. The EnforceSorting rule takes a + // boolean flag indicating whether we elect to transform CoalescePartitionsExec + SortExec + // cascades into SortExec + SortPreservingMergeExec cascades, which enables us to perform + // sorting parallel. + Arc::new(EnforceSorting::new(true)), // The CoalesceBatches rule will not influence the distribution and ordering of the // whole plan tree. Therefore, to avoid influencing other rules, it should run last. Arc::new(CoalesceBatches::new()), diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs index c6c2bd40e39f7..eaa1b6ab07d51 100644 --- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs @@ -1134,7 +1134,7 @@ mod tests { // `EnforceSorting` and `EnfoceDistribution`. // TODO: Orthogonalize the tests here just to verify `EnforceDistribution` and create // new tests for the cascade. - let optimizer = EnforceSorting {}; + let optimizer = EnforceSorting::new(true); let optimized = optimizer.optimize(optimized, &config)?; // Now format correctly diff --git a/datafusion/core/src/physical_optimizer/repartition.rs b/datafusion/core/src/physical_optimizer/repartition.rs index 1285b9089c8d8..e2700ac81dd89 100644 --- a/datafusion/core/src/physical_optimizer/repartition.rs +++ b/datafusion/core/src/physical_optimizer/repartition.rs @@ -461,7 +461,7 @@ mod tests { Arc::new(EnforceDistribution::new()), // EnforceSorting is an essential rule to be applied. // Otherwise, the correctness of the generated optimized plan cannot be guaranteed - Arc::new(EnforceSorting::new()), + Arc::new(EnforceSorting::new(true)), ]; let optimized = optimizers.into_iter().fold($PLAN, |plan, optimizer| { optimizer.optimize(plan, &config).unwrap() diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index c9a3c8fec293e..100f4566db6f8 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -33,10 +33,12 @@ use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::utils::add_sort_above_child; use crate::physical_optimizer::PhysicalOptimizerRule; +use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::rewrite::TreeNodeRewritable; use crate::physical_plan::sorts::sort::SortExec; +use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; -use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; +use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; use arrow::datatypes::SchemaRef; use datafusion_common::{reverse_sort_options, DataFusionError}; use datafusion_physical_expr::utils::{ordering_satisfy, ordering_satisfy_concrete}; @@ -47,14 +49,19 @@ use std::iter::zip; use std::sync::Arc; /// This rule inspects SortExec's in the given physical plan and removes the -/// ones it can prove unnecessary. +/// ones it can prove unnecessary. The boolean flag `parallelize_sorts` +/// indicates whether we elect to transform CoalescePartitionsExec + SortExec +/// cascades into SortExec + SortPreservingMergeExec cascades, which enables +/// us to perform sorting parallel. #[derive(Default)] -pub struct EnforceSorting {} +pub struct EnforceSorting { + parallelize_sorts: bool, +} impl EnforceSorting { #[allow(missing_docs)] - pub fn new() -> Self { - Self {} + pub fn new(parallelize_sorts: bool) -> Self { + Self { parallelize_sorts } } } @@ -100,36 +107,96 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { .into_iter() .map(transform) .collect::>>()?; - let children_plans = children_requirements - .iter() - .map(|elem| elem.plan.clone()) - .collect::>(); let sort_onwards = children_requirements .iter() .map(|item| { - let onwards = &item.sort_onwards; - if !onwards.is_empty() { - let flags = item.plan.maintains_input_order(); - // `onwards` starts from sort introducing executor(e.g `SortExec`, `SortPreservingMergeExec`) till the current executor - // if the executors in between maintain input ordering. If we are at - // the beginning both `SortExec` and `SortPreservingMergeExec` doesn't maintain ordering(they introduce ordering). - // However, we want to propagate them above anyway. - for (maintains, element) in flags.into_iter().zip(onwards.iter()) - { - if (maintains || is_sort(&item.plan)) && !element.is_empty() { - return element.clone(); - } + let flags = item.plan.maintains_input_order(); + // The `sort_onwards` list starts from the sort-introducing operator + // (e.g `SortExec`, `SortPreservingMergeExec`) and collects all the + // intermediate executors that maintain this ordering. If we are at + // the beginning, both `SortExec` and `SortPreservingMergeExec` doesn't + // maintain ordering (as they introduce the ordering). However, we want + // to propagate them above anyway. + for (maintains, element) in + flags.into_iter().zip(item.sort_onwards.iter()) + { + if (maintains || is_sort(&item.plan)) && !element.is_empty() { + return element.clone(); } } vec![] }) .collect::>(); + let children_plans = children_requirements + .iter() + .map(|item| item.plan.clone()) + .collect::>(); let plan = with_new_children_if_necessary(self.plan, children_plans)?; Ok(PlanWithCorrespondingSort { plan, sort_onwards }) } } } +/// This is a "data class" we use within the [EnforceSorting] rule that +/// tracks the closest `CoalescePartitionsExec` descendant of a plan. +#[derive(Debug, Clone)] +struct PlanWithCorrespondingCoalescePartitions { + plan: Arc, + // Keep a vector of `ExecutionPlan`s starting from the closest + // `CoalescePartitionsExec` till the current plan. Since we are sure + // that a `CoalescePartitionsExec` can only be propagated on executors + // with a single child, we use a single vector (not one for each child). + coalesce_onwards: Vec>, +} + +impl PlanWithCorrespondingCoalescePartitions { + pub fn new(plan: Arc) -> Self { + PlanWithCorrespondingCoalescePartitions { + plan, + coalesce_onwards: vec![], + } + } + + pub fn children(&self) -> Vec { + self.plan + .children() + .into_iter() + .map(|child| PlanWithCorrespondingCoalescePartitions::new(child)) + .collect() + } +} + +impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { + fn map_children(self, transform: F) -> Result + where + F: FnMut(Self) -> Result, + { + let children = self.children(); + if children.is_empty() { + Ok(self) + } else { + let children_requirements = children + .into_iter() + .map(transform) + .collect::>>()?; + let coalesce_onwards = children_requirements + .iter() + .map(|item| item.coalesce_onwards.clone()) + .next() + .unwrap(); + let children_plans = children_requirements + .iter() + .map(|item| item.plan.clone()) + .collect::>(); + let plan = with_new_children_if_necessary(self.plan, children_plans)?; + Ok(PlanWithCorrespondingCoalescePartitions { + plan, + coalesce_onwards, + }) + } + } +} + impl PhysicalOptimizerRule for EnforceSorting { fn optimize( &self, @@ -139,7 +206,15 @@ impl PhysicalOptimizerRule for EnforceSorting { // Execute a post-order traversal to adjust input key ordering: let plan_requirements = PlanWithCorrespondingSort::new(plan); let adjusted = plan_requirements.transform_up(&ensure_sorting)?; - Ok(adjusted.plan) + if self.parallelize_sorts { + let plan_with_coalesce_partitions = + PlanWithCorrespondingCoalescePartitions::new(adjusted.plan); + let parallel = + plan_with_coalesce_partitions.transform_up(¶llelize_sorts)?; + Ok(parallel.plan) + } else { + Ok(adjusted.plan) + } } fn name(&self) -> &str { @@ -151,6 +226,89 @@ impl PhysicalOptimizerRule for EnforceSorting { } } +/// This function turns plans of the form +/// "SortExec: [a@0 ASC]", +/// " CoalescePartitionsExec", +/// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", +/// to +/// "SortPreservingMergeExec: [a@0 ASC]", +/// " SortExec: [a@0 ASC]", +/// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", +/// by following connections from `CoalescePartitionsExec`s to `SortExec`s. +/// By performing sorting in parallel, we can increase performance in some scenarios. +fn parallelize_sorts( + requirements: PlanWithCorrespondingCoalescePartitions, +) -> Result> { + let plan = requirements.plan; + if plan.children().is_empty() { + return Ok(None); + } + let requires_single_partition = matches!( + plan.required_input_distribution()[0], + Distribution::SinglePartition + ); + let mut coalesce_onwards = requirements.coalesce_onwards.clone(); + if let Some(sort_exec) = plan.as_any().downcast_ref::() { + let sort_expr = sort_exec.expr(); + // If there is link between CoalescePartitionsExec and SortExec that satisfy requirements + // (e.g all Executors in between doesn't require to work on SinglePartition). + // We can replace CoalescePartitionsExec with SortExec and SortExec with SortPreservingMergeExec + // respectively to parallelize sorting. + if !coalesce_onwards.is_empty() { + let coalesce_partitions_any = coalesce_onwards[0].clone(); + let mut prev_layer = coalesce_partitions_any.children()[0].clone(); + if !ordering_satisfy(prev_layer.output_ordering(), Some(sort_expr), || { + sort_exec.equivalence_properties() + }) { + prev_layer = add_sort_above_child(&prev_layer, sort_expr.to_vec())?; + }; + // In the loop below, we start from one as the first entry is a + // `CoalescePartitionsExec` and we are removing it from the plan. + for layer in coalesce_onwards.iter().skip(1) { + let mut children = layer.children(); + children[0] = prev_layer; + prev_layer = layer.clone().with_new_children(children)?; + } + let spm = SortPreservingMergeExec::new(sort_expr.to_vec(), prev_layer); + return Ok(Some(PlanWithCorrespondingCoalescePartitions { + plan: Arc::new(spm), + coalesce_onwards: vec![], + })); + } + } else if plan.as_any().is::() { + // There is a CoalescePartitions that is unnecesary, remove it. + if !coalesce_onwards.is_empty() { + let coalesce_partitions_any = coalesce_onwards[0].clone(); + let mut prev_layer = coalesce_partitions_any.children()[0].clone(); + for layer in coalesce_onwards.iter().skip(1) { + let mut children = layer.children(); + children[0] = prev_layer; + prev_layer = layer.clone().with_new_children(children)?; + } + let new_plan = plan.clone().with_new_children(vec![prev_layer])?; + return Ok(Some(PlanWithCorrespondingCoalescePartitions { + plan: new_plan.clone(), + coalesce_onwards: vec![new_plan], + })); + } else { + // starting of coalesce partition + coalesce_onwards = vec![plan.clone()]; + } + } else if !requires_single_partition && !coalesce_onwards.is_empty() { + // The Executor above(not necessarily immediately above) CoalescePartitionExec doesn't require SinglePartition + // hence CoalescePartitionExec is not a requirement of this executor. + // We can propagate it upwards. + coalesce_onwards.push(plan.clone()); + } else { + coalesce_onwards.clear(); + } + + Ok(Some(PlanWithCorrespondingCoalescePartitions { + plan, + coalesce_onwards, + })) +} + // Checks whether executor is Sort // TODO: Add support for SortPreservingMergeExec also. fn is_sort(plan: &Arc) -> bool { @@ -192,24 +350,29 @@ fn ensure_sorting( if let [first, ..] = sort_onwards.as_slice() { // The ordering requirement is met, we can analyze if there is an unnecessary sort: let sort_any = first.1.clone(); - let sort_exec = convert_to_sort_exec(&sort_any)?; - let sort_output_ordering = sort_exec.output_ordering(); - let sort_input_ordering = sort_exec.input().output_ordering(); + let sort_output_ordering = sort_any.output_ordering(); + // Variable `sort_any` will either be a `SortExec` or a + // `SortPreservingMergeExec`, and both have single child. + // Therefore, we can use the 0th index without loss of generality. + let sort_input = sort_any.children()[0].clone(); + let sort_input_ordering = sort_input.output_ordering(); // Simple analysis: Does the input of the sort in question already satisfy the ordering requirements? if ordering_satisfy(sort_input_ordering, sort_output_ordering, || { - sort_exec.input().equivalence_properties() + sort_input.equivalence_properties() }) { update_child_to_remove_unnecessary_sort(child, sort_onwards)?; } // For window expressions, we can remove some sorts when we can // calculate the result in reverse: else if let Some(exec) = - requirements.plan.as_any().downcast_ref::() + plan.as_any().downcast_ref::() { if let Some(result) = analyze_window_sort_removal( exec.window_expr(), &exec.partition_keys, - sort_exec, + &sort_input, + sort_output_ordering, + sort_input_ordering, sort_onwards, )? { return Ok(Some(result)); @@ -222,7 +385,9 @@ fn ensure_sorting( if let Some(result) = analyze_window_sort_removal( exec.window_expr(), &exec.partition_keys, - sort_exec, + &sort_input, + sort_output_ordering, + sort_input_ordering, sort_onwards, )? { return Ok(Some(result)); @@ -243,8 +408,13 @@ fn ensure_sorting( (None, Some(_)) => { // We have a SortExec whose effect may be neutralized by a order-imposing // operator. In this case, remove this sort: - if !requirements.plan.maintains_input_order()[idx] { - update_child_to_remove_unnecessary_sort(child, sort_onwards)?; + if !plan.maintains_input_order()[idx] { + if plan.output_ordering().is_some() && !is_sort(plan) { + let count = plan.output_ordering().unwrap().len(); + update_child_to_change_finer_sort(child, sort_onwards, count)?; + } else { + update_child_to_remove_unnecessary_sort(child, sort_onwards)?; + } } } (None, None) => {} @@ -291,14 +461,30 @@ fn analyze_immediate_sort_removal( sort_exec.output_ordering(), || sort_exec.input().equivalence_properties(), ) { + let sort_input = sort_exec.input(); // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: let mut new_onwards = requirements.sort_onwards[0].to_vec(); if !new_onwards.is_empty() { new_onwards.pop(); } + let updated_plan = if !sort_exec.preserve_partitioning() + && sort_input.output_partitioning().partition_count() > 1 + { + // Replace the sort with a sort-preserving merge: + let new_plan: Arc = + Arc::new(SortPreservingMergeExec::new( + sort_exec.expr().to_vec(), + sort_input.clone(), + )); + new_onwards.push((0, new_plan.clone())); + new_plan + } else { + // Remove the sort: + sort_input.clone() + }; return Ok(Some(PlanWithCorrespondingSort { - plan: sort_exec.input().clone(), + plan: updated_plan, sort_onwards: vec![new_onwards], })); } @@ -311,13 +497,14 @@ fn analyze_immediate_sort_removal( fn analyze_window_sort_removal( window_expr: &[Arc], partition_keys: &[Arc], - sort_exec: &SortExec, + sort_input: &Arc, + sort_output_ordering: Option<&[PhysicalSortExpr]>, + physical_ordering: Option<&[PhysicalSortExpr]>, sort_onward: &mut Vec<(usize, Arc)>, ) -> Result> { - let required_ordering = sort_exec.output_ordering().ok_or_else(|| { + let required_ordering = sort_output_ordering.ok_or_else(|| { DataFusionError::Plan("A SortExec should have output ordering".to_string()) })?; - let physical_ordering = sort_exec.input().output_ordering(); let physical_ordering = if let Some(physical_ordering) = physical_ordering { physical_ordering } else { @@ -327,7 +514,7 @@ fn analyze_window_sort_removal( let (can_skip_sorting, should_reverse) = can_skip_sort( window_expr[0].partition_by(), required_ordering, - &sort_exec.input().schema(), + &sort_input.schema(), physical_ordering, )?; if can_skip_sorting { @@ -379,11 +566,32 @@ fn update_child_to_remove_unnecessary_sort( Ok(()) } -/// Converts an [ExecutionPlan] trait object to a [SortExec] when possible. -fn convert_to_sort_exec(sort_any: &Arc) -> Result<&SortExec> { - sort_any.as_any().downcast_ref::().ok_or_else(|| { - DataFusionError::Plan("Given ExecutionPlan is not a SortExec".to_string()) - }) +/// Updates child to modify the unnecessarily fine sorting below it. +fn update_child_to_change_finer_sort( + child: &mut Arc, + sort_onwards: &mut Vec<(usize, Arc)>, + n_sort_expr: usize, +) -> Result<()> { + if !sort_onwards.is_empty() { + *child = change_finer_sort_in_sub_plan(sort_onwards, n_sort_expr)?; + } + Ok(()) +} + +/// Converts an [ExecutionPlan] trait object to a [PhysicalSortExpr] slice when possible. +fn get_sort_exprs(sort_any: &Arc) -> Result<&[PhysicalSortExpr]> { + if let Some(sort_exec) = sort_any.as_any().downcast_ref::() { + Ok(sort_exec.expr()) + } else if let Some(sort_preserving_merge_exec) = + sort_any.as_any().downcast_ref::() + { + Ok(sort_preserving_merge_exec.expr()) + } else { + Err(DataFusionError::Plan( + "Given ExecutionPlan is not a SortExec or a SortPreservingMergeExec" + .to_string(), + )) + } } /// Removes the sort from the plan in `sort_onwards`. @@ -391,10 +599,9 @@ fn remove_corresponding_sort_from_sub_plan( sort_onwards: &mut Vec<(usize, Arc)>, ) -> Result> { let (_, sort_any) = sort_onwards[0].clone(); - let sort_exec = convert_to_sort_exec(&sort_any)?; - let mut prev_layer = sort_exec.input().clone(); - // In the loop below, se start from 1 as the first one is a SortExec - // and we are removing it from the plan. + let mut prev_layer = sort_any.children()[0].clone(); + // In the loop below, we start from one as the first entry is a + // `SortExec` and we are removing it from the plan. for (child_idx, layer) in sort_onwards.iter().skip(1) { let mut children = layer.children(); children[*child_idx] = prev_layer; @@ -405,6 +612,26 @@ fn remove_corresponding_sort_from_sub_plan( Ok(prev_layer) } +/// Change the unnecessarily fine sort in `sort_onwards`. +fn change_finer_sort_in_sub_plan( + sort_onwards: &mut [(usize, Arc)], + n_sort_expr: usize, +) -> Result> { + let (sort_child_idx, sort_any) = sort_onwards[0].clone(); + let new_sort_expr = get_sort_exprs(&sort_any)?[0..n_sort_expr].to_vec(); + let mut prev_layer = sort_any.children()[0].clone(); + let updated_sort = add_sort_above_child(&prev_layer, new_sort_expr)?; + sort_onwards[0] = (sort_child_idx, updated_sort); + // In the loop below, we start from one as the first entry is a + // `SortExec` and we are removing it from the plan. + for (child_idx, layer) in sort_onwards.iter() { + let mut children = layer.children(); + children[*child_idx] = prev_layer; + prev_layer = layer.clone().with_new_children(children)?; + } + Ok(prev_layer) +} + #[derive(Debug)] /// This structure stores extra column information required to remove unnecessary sorts. pub struct ColumnInfo { @@ -498,13 +725,15 @@ mod tests { use super::*; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::physical_plan::displayable; + use crate::physical_optimizer::dist_enforcement::EnforceDistribution; use crate::physical_plan::file_format::{FileScanConfig, ParquetExec}; use crate::physical_plan::filter::FilterExec; use crate::physical_plan::memory::MemoryExec; + use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::create_window_expr; + use crate::physical_plan::{displayable, Partitioning}; use crate::prelude::SessionContext; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -522,6 +751,13 @@ mod tests { Ok(schema) } + // Util function to get string representation of a physical plan + fn get_plan_string(plan: &Arc) -> Vec { + let formatted = displayable(plan.as_ref()).indent().to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + actual.iter().map(|elem| elem.to_string()).collect() + } + #[tokio::test] async fn test_is_column_aligned_nullable() -> Result<()> { let schema = create_test_schema()?; @@ -632,12 +868,9 @@ mod tests { // Run the actual optimizer let optimized_physical_plan = - EnforceSorting::new().optimize(physical_plan, state.config_options())?; - - let formatted = displayable(optimized_physical_plan.as_ref()) - .indent() - .to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); + EnforceSorting::new(true).optimize(physical_plan, state.config_options())?; + // Get string representation of the plan + let actual = get_plan_string(&optimized_physical_plan); assert_eq!( expected_optimized_lines, actual, "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected_optimized_lines:#?}\nactual:\n\n{actual:#?}\n\n" @@ -897,6 +1130,103 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_union_inputs_different_sorted3() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + let sort2 = sort_exec(sort_exprs2, source1); + + let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); + + let union = union_exec(vec![sort1, source2, sort2]); + let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); + + // First input to the union is not Sorted (SortExec is finer than required ordering by the SortPreservingMergeExec above). + // Second input to the union is already Sorted (matches with the required ordering by the SortPreservingMergeExec above). + // Third input to the union is not Sorted (SortExec is matches required ordering by the SortPreservingMergeExec above). + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + // should adjust sorting in the first input of the union such that it is not unnecessarily fine + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + // With new change in SortEnforcement EnforceSorting->EnforceDistribution->EnforceSorting + // should produce same result with EnforceDistribution+EnforceSorting + // This enables us to use EnforceSorting possibly before EnforceDistribution + // Given that it will be called at least once after last EnforceDistribution. The reason is that + // EnforceDistribution may invalidate ordering invariant. + async fn test_commutativity() -> Result<()> { + let schema = create_test_schema()?; + + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + + let memory_exec = memory_exec(&schema); + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let window = window_exec("nullable_col", sort_exprs.clone(), memory_exec); + let repartition = Arc::new(RepartitionExec::try_new( + window, + Partitioning::RoundRobinBatch(2), + )?) as Arc; + + let orig_plan = Arc::new(SortExec::new_with_partitioning( + sort_exprs, + repartition, + false, + None, + )) as Arc; + + let mut plan = orig_plan.clone(); + let rules = vec![ + Arc::new(EnforceDistribution::new()) as Arc, + Arc::new(EnforceSorting::new(true)) as Arc, + ]; + for rule in rules { + plan = rule.optimize(plan, state.config_options())?; + } + let first_plan = plan.clone(); + + let mut plan = orig_plan.clone(); + let rules = vec![ + Arc::new(EnforceSorting::new(true)) as Arc, + Arc::new(EnforceDistribution::new()) as Arc, + Arc::new(EnforceSorting::new(true)) as Arc, + ]; + for rule in rules { + plan = rule.optimize(plan, state.config_options())?; + } + let second_plan = plan.clone(); + + assert_eq!(get_plan_string(&first_plan), get_plan_string(&second_plan)); + Ok(()) + } + /// make PhysicalSortExpr with default options fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { sort_expr_options(name, schema, SortOptions::default()) diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 1404dfa20c30c..8689b016b01c1 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -118,7 +118,12 @@ impl QueryCase { if error.is_some() { let plan_error = plan.unwrap_err(); let initial = error.unwrap().to_string(); - assert!(plan_error.to_string().contains(initial.as_str())); + assert!( + plan_error.to_string().contains(initial.as_str()), + "plan_error: {:?} doesn't contain message: {:?}", + plan_error, + initial.as_str() + ); } else { assert!(plan.is_ok()) } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 01bd94e8e4bbe..5fc877a2c8e65 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -86,11 +86,6 @@ async fn explain_analyze_baseline_metrics() { "CoalesceBatchesExec: target_batch_size=4096", "metrics=[output_rows=5, elapsed_compute" ); - assert_metrics!( - &formatted, - "CoalescePartitionsExec", - "metrics=[output_rows=5, elapsed_compute=" - ); assert_metrics!( &formatted, "UnionExec", diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index e0bd1a523c4ad..37b662a284c9b 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -1980,8 +1980,8 @@ async fn left_semi_join() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST]", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", @@ -1997,8 +1997,8 @@ async fn left_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST]", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", @@ -2062,8 +2062,8 @@ async fn left_semi_join() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST]", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", @@ -2078,8 +2078,8 @@ async fn left_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST]", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " CoalesceBatchesExec: target_batch_size=4096", @@ -2259,8 +2259,8 @@ async fn right_semi_join() -> Result<()> { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { - vec![ "SortExec: [t1_id@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST]", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 1 }, op: NotEq, right: Column { name: \"t1_name\", index: 0 } }", @@ -2275,8 +2275,8 @@ async fn right_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST]", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " CoalesceBatchesExec: target_batch_size=4096", @@ -2307,8 +2307,8 @@ async fn right_semi_join() -> Result<()> { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { - vec![ "SortExec: [t1_id@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST]", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 0 }, op: NotEq, right: Column { name: \"t1_name\", index: 1 } }", @@ -2323,8 +2323,8 @@ async fn right_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST]", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " CoalesceBatchesExec: target_batch_size=4096", diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index f0fd04efd1de6..b7742a2c98033 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -1746,8 +1746,8 @@ async fn over_order_by_sort_keys_sorting_global_order_compacting() -> Result<()> // 3 SortExec are added let expected = { vec![ - "SortExec: [c2@0 ASC NULLS LAST]", - " CoalescePartitionsExec", + "SortPreservingMergeExec: [c2@0 ASC NULLS LAST]", + " SortExec: [c2@0 ASC NULLS LAST]", " ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9), SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9), MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9)]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", @@ -2297,15 +2297,14 @@ async fn test_remove_unnecessary_sort_in_sub_query() -> Result<()> { " AggregateExec: mode=Final, gby=[], aggr=[COUNT(UInt8(1))]", " CoalescePartitionsExec", " AggregateExec: mode=Partial, gby=[], aggr=[COUNT(UInt8(1))]", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CoalescePartitionsExec", - " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 8), input_partitions=8", - " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]", - " CoalesceBatchesExec: target_batch_size=4096", - " FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=8", + " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 8), input_partitions=8", + " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(UInt8(1))]", + " CoalesceBatchesExec: target_batch_size=4096", + " FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ] }; @@ -2384,6 +2383,41 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re Ok(()) } +#[tokio::test] +async fn test_window_agg_global_sort() -> Result<()> { + let config = SessionConfig::new() + .with_repartition_windows(false) + .with_target_partitions(2); + let ctx = SessionContext::with_config(config); + register_aggregate_csv(&ctx).await?; + let sql = "SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + let formatted = displayable(physical_plan.as_ref()).indent().to_string(); + // Only 1 SortExec was added + let expected = { + vec![ + "SortPreservingMergeExec: [c1@0 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]", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", 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)) }]", + " SortExec: [c1@0 ASC NULLS LAST]", + ] + }; + + let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual_len = actual.len(); + let actual_trim_last = &actual[..actual_len - 1]; + assert_eq!( + expected, actual_trim_last, + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + Ok(()) +} + fn write_test_data_to_parquet(tmpdir: &TempDir, n_file: usize) -> Result<()> { let ts_field = Field::new("ts", DataType::Int32, false); let inc_field = Field::new("inc_col", DataType::Int32, false); From fd7a7147e07438358605d78f4bcedaee0b24f00d Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 25 Jan 2023 21:45:52 -0600 Subject: [PATCH 02/31] Remove unnecessary cloning --- .../physical_optimizer/sort_enforcement.rs | 36 +++++++++---------- 1 file changed, 16 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 100f4566db6f8..435e7f2d65d7a 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -247,16 +247,15 @@ fn parallelize_sorts( plan.required_input_distribution()[0], Distribution::SinglePartition ); - let mut coalesce_onwards = requirements.coalesce_onwards.clone(); + let mut coalesce_onwards = requirements.coalesce_onwards; if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_expr = sort_exec.expr(); // If there is link between CoalescePartitionsExec and SortExec that satisfy requirements // (e.g all Executors in between doesn't require to work on SinglePartition). // We can replace CoalescePartitionsExec with SortExec and SortExec with SortPreservingMergeExec // respectively to parallelize sorting. - if !coalesce_onwards.is_empty() { - let coalesce_partitions_any = coalesce_onwards[0].clone(); - let mut prev_layer = coalesce_partitions_any.children()[0].clone(); + if let [first, ..] = coalesce_onwards.as_slice() { + let mut prev_layer = first.children()[0].clone(); if !ordering_satisfy(prev_layer.output_ordering(), Some(sort_expr), || { sort_exec.equivalence_properties() }) { @@ -277,9 +276,8 @@ fn parallelize_sorts( } } else if plan.as_any().is::() { // There is a CoalescePartitions that is unnecesary, remove it. - if !coalesce_onwards.is_empty() { - let coalesce_partitions_any = coalesce_onwards[0].clone(); - let mut prev_layer = coalesce_partitions_any.children()[0].clone(); + if let [first, ..] = coalesce_onwards.as_slice() { + let mut prev_layer = first.children()[0].clone(); for layer in coalesce_onwards.iter().skip(1) { let mut children = layer.children(); children[0] = prev_layer; @@ -455,16 +453,16 @@ fn analyze_immediate_sort_removal( requirements: &PlanWithCorrespondingSort, ) -> Result> { if let Some(sort_exec) = requirements.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_exec.input().output_ordering(), + sort_input.output_ordering(), sort_exec.output_ordering(), - || sort_exec.input().equivalence_properties(), + || sort_input.equivalence_properties(), ) { - let sort_input = sort_exec.input(); // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: - let mut new_onwards = requirements.sort_onwards[0].to_vec(); + let mut new_onwards = requirements.sort_onwards[0].clone(); if !new_onwards.is_empty() { new_onwards.pop(); } @@ -472,16 +470,14 @@ fn analyze_immediate_sort_removal( && sort_input.output_partitioning().partition_count() > 1 { // Replace the sort with a sort-preserving merge: - let new_plan: Arc = - Arc::new(SortPreservingMergeExec::new( - sort_exec.expr().to_vec(), - sort_input.clone(), - )); + let new_plan: Arc = Arc::new( + SortPreservingMergeExec::new(sort_exec.expr().to_vec(), sort_input), + ); new_onwards.push((0, new_plan.clone())); new_plan } else { // Remove the sort: - sort_input.clone() + sort_input }; return Ok(Some(PlanWithCorrespondingSort { plan: updated_plan, @@ -598,7 +594,7 @@ fn get_sort_exprs(sort_any: &Arc) -> Result<&[PhysicalSortExp fn remove_corresponding_sort_from_sub_plan( sort_onwards: &mut Vec<(usize, Arc)>, ) -> Result> { - let (_, sort_any) = sort_onwards[0].clone(); + let (_, sort_any) = &sort_onwards[0]; let mut prev_layer = sort_any.children()[0].clone(); // In the loop below, we start from one as the first entry is a // `SortExec` and we are removing it from the plan. @@ -617,8 +613,8 @@ fn change_finer_sort_in_sub_plan( sort_onwards: &mut [(usize, Arc)], n_sort_expr: usize, ) -> Result> { - let (sort_child_idx, sort_any) = sort_onwards[0].clone(); - let new_sort_expr = get_sort_exprs(&sort_any)?[0..n_sort_expr].to_vec(); + let (sort_child_idx, ref sort_any) = sort_onwards[0]; + let new_sort_expr = get_sort_exprs(sort_any)?[0..n_sort_expr].to_vec(); let mut prev_layer = sort_any.children()[0].clone(); let updated_sort = add_sort_above_child(&prev_layer, new_sort_expr)?; sort_onwards[0] = (sort_child_idx, updated_sort); From 015f931666fe19bb512eea07e96181dc92c6ae3a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 30 Jan 2023 15:44:03 +0300 Subject: [PATCH 03/31] Convert sort_onward to tree to support multipath --- .../physical_optimizer/sort_enforcement.rs | 219 ++++++++++++------ 1 file changed, 152 insertions(+), 67 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 435e7f2d65d7a..186d8863125e8 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -37,6 +37,7 @@ use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::rewrite::TreeNodeRewritable; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; use arrow::datatypes::SchemaRef; @@ -65,6 +66,27 @@ impl EnforceSorting { } } +#[derive(Debug, Clone)] +struct ExecTree { + /// child index of the plan in its parent executor + pub idx: usize, + /// Children of the `plan` that needs to be updated if bottom executor is removed + pub children: Vec, + /// ExecutionPlan node of the tree + pub plan: Arc, +} + +impl ExecTree { + /// Executor at the leaf of the tree + fn bottom_executor(&self) -> Arc { + if !self.children.is_empty() { + self.children[0].bottom_executor() + } else { + self.plan.clone() + } + } +} + /// This is a "data class" we use within the [EnforceSorting] rule that /// tracks the closest `SortExec` descendant for every child of a plan. #[derive(Debug, Clone)] @@ -73,7 +95,7 @@ struct PlanWithCorrespondingSort { // For every child, keep a vector of `ExecutionPlan`s starting from the // closest `SortExec` till the current plan. The first index of the tuple is // the child index of the plan -- we need this information as we make updates. - sort_onwards: Vec)>>, + sort_onwards: Vec>, } impl PlanWithCorrespondingSort { @@ -81,7 +103,7 @@ impl PlanWithCorrespondingSort { let length = plan.children().len(); PlanWithCorrespondingSort { plan, - sort_onwards: vec![vec![]; length], + sort_onwards: vec![None; length], } } @@ -109,22 +131,44 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { .collect::>>()?; let sort_onwards = children_requirements .iter() - .map(|item| { + .enumerate() + .map(|(idx, item)| { let flags = item.plan.maintains_input_order(); + let required_ordering = item.plan.required_input_ordering(); // The `sort_onwards` list starts from the sort-introducing operator // (e.g `SortExec`, `SortPreservingMergeExec`) and collects all the // intermediate executors that maintain this ordering. If we are at // the beginning, both `SortExec` and `SortPreservingMergeExec` doesn't // maintain ordering (as they introduce the ordering). However, we want // to propagate them above anyway. - for (maintains, element) in - flags.into_iter().zip(item.sort_onwards.iter()) + let mut res = vec![]; + for (maintains, element, required_ordering) in + izip!(flags, item.sort_onwards.clone(), required_ordering) { - if (maintains || is_sort(&item.plan)) && !element.is_empty() { - return element.clone(); + if (maintains || item.plan.as_any().is::()) + && element.is_some() + && required_ordering.is_none() + { + // return element.clone(); + res.push(element.clone().unwrap()); } } - vec![] + // None + if !res.is_empty() { + Some(ExecTree { + idx, + plan: item.plan.clone(), + children: res, + }) + } else if is_sort(&item.plan) { + Some(ExecTree { + idx, + plan: item.plan.clone(), + children: vec![], + }) + } else { + None + } }) .collect::>(); let children_plans = children_requirements @@ -343,11 +387,16 @@ fn ensure_sorting( update_child_to_remove_unnecessary_sort(child, sort_onwards)?; let sort_expr = required_ordering.to_vec(); *child = add_sort_above_child(child, sort_expr)?; - sort_onwards.push((idx, child.clone())) + // sort_onwards.push((idx, child.clone())) + *sort_onwards = Some(ExecTree { + idx, + plan: child.clone(), + children: vec![], + }) } - if let [first, ..] = sort_onwards.as_slice() { + if let Some(sort_onward) = sort_onwards { // The ordering requirement is met, we can analyze if there is an unnecessary sort: - let sort_any = first.1.clone(); + let sort_any = sort_onward.bottom_executor(); let sort_output_ordering = sort_any.output_ordering(); // Variable `sort_any` will either be a `SortExec` or a // `SortPreservingMergeExec`, and both have single child. @@ -371,7 +420,7 @@ fn ensure_sorting( &sort_input, sort_output_ordering, sort_input_ordering, - sort_onwards, + sort_onward, )? { return Ok(Some(result)); } @@ -386,7 +435,7 @@ fn ensure_sorting( &sort_input, sort_output_ordering, sort_input_ordering, - sort_onwards, + sort_onward, )? { return Ok(Some(result)); } @@ -401,7 +450,11 @@ fn ensure_sorting( // Ordering requirement is not met, we should add a SortExec to the plan. let sort_expr = required.to_vec(); *child = add_sort_above_child(child, sort_expr)?; - *sort_onwards = vec![(idx, child.clone())]; + *sort_onwards = Some(ExecTree { + idx, + plan: child.clone(), + children: vec![], + }) } (None, Some(_)) => { // We have a SortExec whose effect may be neutralized by a order-imposing @@ -422,24 +475,6 @@ fn ensure_sorting( Ok(Some(requirements)) } else { let new_plan = requirements.plan.with_new_children(new_children)?; - for (idx, (trace, required_ordering)) in new_onwards - .iter_mut() - .zip(new_plan.required_input_ordering()) - .enumerate() - .take(new_plan.children().len()) - { - if new_plan.maintains_input_order()[idx] - && required_ordering.is_none() - && !trace.is_empty() - { - trace.push((idx, new_plan.clone())); - } else { - trace.clear(); - if is_sort(&new_plan) { - trace.push((idx, new_plan.clone())); - } - } - } Ok(Some(PlanWithCorrespondingSort { plan: new_plan, sort_onwards: new_onwards, @@ -463,8 +498,9 @@ fn analyze_immediate_sort_removal( // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: let mut new_onwards = requirements.sort_onwards[0].clone(); - if !new_onwards.is_empty() { - new_onwards.pop(); + let mut children = vec![]; + if let Some(tmp) = &new_onwards { + children.push(tmp.clone()); } let updated_plan = if !sort_exec.preserve_partitioning() && sort_input.output_partitioning().partition_count() > 1 @@ -473,7 +509,12 @@ fn analyze_immediate_sort_removal( let new_plan: Arc = Arc::new( SortPreservingMergeExec::new(sort_exec.expr().to_vec(), sort_input), ); - new_onwards.push((0, new_plan.clone())); + // new_onwards.push((0, new_plan.clone())); + new_onwards = Some(ExecTree { + idx: 0, + plan: new_plan.clone(), + children, + }); new_plan } else { // Remove the sort: @@ -496,7 +537,7 @@ fn analyze_window_sort_removal( sort_input: &Arc, sort_output_ordering: Option<&[PhysicalSortExpr]>, physical_ordering: Option<&[PhysicalSortExpr]>, - sort_onward: &mut Vec<(usize, Arc)>, + sort_onward: &mut ExecTree, ) -> Result> { let required_ordering = sort_output_ordering.ok_or_else(|| { DataFusionError::Plan("A SortExec should have output ordering".to_string()) @@ -554,9 +595,9 @@ fn analyze_window_sort_removal( /// Updates child to remove the unnecessary sorting below it. fn update_child_to_remove_unnecessary_sort( child: &mut Arc, - sort_onwards: &mut Vec<(usize, Arc)>, + sort_onwards: &mut Option, ) -> Result<()> { - if !sort_onwards.is_empty() { + if let Some(sort_onwards) = sort_onwards { *child = remove_corresponding_sort_from_sub_plan(sort_onwards)?; } Ok(()) @@ -565,10 +606,10 @@ fn update_child_to_remove_unnecessary_sort( /// Updates child to modify the unnecessarily fine sorting below it. fn update_child_to_change_finer_sort( child: &mut Arc, - sort_onwards: &mut Vec<(usize, Arc)>, + sort_onwards: &mut Option, n_sort_expr: usize, ) -> Result<()> { - if !sort_onwards.is_empty() { + if let Some(sort_onwards) = sort_onwards { *child = change_finer_sort_in_sub_plan(sort_onwards, n_sort_expr)?; } Ok(()) @@ -592,40 +633,41 @@ fn get_sort_exprs(sort_any: &Arc) -> Result<&[PhysicalSortExp /// Removes the sort from the plan in `sort_onwards`. fn remove_corresponding_sort_from_sub_plan( - sort_onwards: &mut Vec<(usize, Arc)>, + sort_onwards: &mut ExecTree, ) -> Result> { - let (_, sort_any) = &sort_onwards[0]; - let mut prev_layer = sort_any.children()[0].clone(); - // In the loop below, we start from one as the first entry is a - // `SortExec` and we are removing it from the plan. - for (child_idx, layer) in sort_onwards.iter().skip(1) { - let mut children = layer.children(); - children[*child_idx] = prev_layer; - prev_layer = layer.clone().with_new_children(children)?; - } - // We have removed the sort, hence empty the sort_onwards: - sort_onwards.clear(); - Ok(prev_layer) + let res = if is_sort(&sort_onwards.plan) { + sort_onwards.plan.children()[0].clone() + } else { + let res = sort_onwards.plan.clone(); + let mut children = res.children(); + for elem in &mut sort_onwards.children { + children[elem.idx] = remove_corresponding_sort_from_sub_plan(elem)?; + } + res.with_new_children(children)? + }; + + Ok(res) } /// Change the unnecessarily fine sort in `sort_onwards`. fn change_finer_sort_in_sub_plan( - sort_onwards: &mut [(usize, Arc)], + sort_onwards: &mut ExecTree, n_sort_expr: usize, ) -> Result> { - let (sort_child_idx, ref sort_any) = sort_onwards[0]; - let new_sort_expr = get_sort_exprs(sort_any)?[0..n_sort_expr].to_vec(); - let mut prev_layer = sort_any.children()[0].clone(); - let updated_sort = add_sort_above_child(&prev_layer, new_sort_expr)?; - sort_onwards[0] = (sort_child_idx, updated_sort); - // In the loop below, we start from one as the first entry is a - // `SortExec` and we are removing it from the plan. - for (child_idx, layer) in sort_onwards.iter() { - let mut children = layer.children(); - children[*child_idx] = prev_layer; - prev_layer = layer.clone().with_new_children(children)?; - } - Ok(prev_layer) + let res = if is_sort(&sort_onwards.plan) { + let prev_layer = sort_onwards.plan.children()[0].clone(); + let new_sort_expr = get_sort_exprs(&sort_onwards.plan)?[0..n_sort_expr].to_vec(); + add_sort_above_child(&prev_layer, new_sort_expr)? + } else { + let res = sort_onwards.plan.clone(); + let mut children = res.children(); + for elem in &mut sort_onwards.children { + children[elem.idx] = remove_corresponding_sort_from_sub_plan(elem)?; + } + res.with_new_children(children)? + }; + + Ok(res) } #[derive(Debug)] @@ -1171,6 +1213,49 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_union_inputs_different_sorted4() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs2.clone(), source1.clone()); + let sort2 = sort_exec(sort_exprs2.clone(), source1); + + let source2 = parquet_exec_sorted(&schema, sort_exprs2); + + let union = union_exec(vec![sort1, source2, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs1, union); + + // First input to the union is not Sorted (SortExec is finer than required ordering by the SortPreservingMergeExec above). + // Second input to the union is already Sorted (matches with the required ordering by the SortPreservingMergeExec above). + // Third input to the union is not Sorted (SortExec is matches required ordering by the SortPreservingMergeExec above). + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + // should adjust sorting in the first input of the union such that it is not unnecessarily fine + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] // With new change in SortEnforcement EnforceSorting->EnforceDistribution->EnforceSorting // should produce same result with EnforceDistribution+EnforceSorting From b35696294500860e6a7a83c279a337670308c9ef Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 31 Jan 2023 11:33:26 +0300 Subject: [PATCH 04/31] Add limit bug handling --- .../physical_optimizer/sort_enforcement.rs | 104 ++++++++++-------- datafusion/core/tests/sql/window.rs | 46 ++++++++ 2 files changed, 104 insertions(+), 46 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 186d8863125e8..e2e43afa23961 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -34,6 +34,7 @@ use crate::error::Result; use crate::physical_optimizer::utils::add_sort_above_child; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::rewrite::TreeNodeRewritable; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -78,11 +79,15 @@ struct ExecTree { impl ExecTree { /// Executor at the leaf of the tree - fn bottom_executor(&self) -> Arc { + fn bottom_executors(&self) -> Vec> { if !self.children.is_empty() { - self.children[0].bottom_executor() + let mut res = vec![]; + for child in &self.children { + res.extend(child.bottom_executors()) + } + res } else { - self.plan.clone() + vec![self.plan.clone()] } } } @@ -148,6 +153,8 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { if (maintains || item.plan.as_any().is::()) && element.is_some() && required_ordering.is_none() + // Sorts under limit are not removable. + && !is_limit(&item.plan) { // return element.clone(); res.push(element.clone().unwrap()); @@ -351,6 +358,11 @@ fn parallelize_sorts( })) } +// Checks whether executor is limit executor (e.g either LocalLimitExec or GlobalLimitExec) +fn is_limit(plan: &Arc) -> bool { + plan.as_any().is::() || plan.as_any().is::() +} + // Checks whether executor is Sort // TODO: Add support for SortPreservingMergeExec also. fn is_sort(plan: &Arc) -> bool { @@ -395,31 +407,12 @@ fn ensure_sorting( }) } if let Some(sort_onward) = sort_onwards { - // The ordering requirement is met, we can analyze if there is an unnecessary sort: - let sort_any = sort_onward.bottom_executor(); - let sort_output_ordering = sort_any.output_ordering(); - // Variable `sort_any` will either be a `SortExec` or a - // `SortPreservingMergeExec`, and both have single child. - // Therefore, we can use the 0th index without loss of generality. - let sort_input = sort_any.children()[0].clone(); - let sort_input_ordering = sort_input.output_ordering(); - // Simple analysis: Does the input of the sort in question already satisfy the ordering requirements? - if ordering_satisfy(sort_input_ordering, sort_output_ordering, || { - sort_input.equivalence_properties() - }) { - update_child_to_remove_unnecessary_sort(child, sort_onwards)?; - } // For window expressions, we can remove some sorts when we can // calculate the result in reverse: - else if let Some(exec) = - plan.as_any().downcast_ref::() - { + if let Some(exec) = plan.as_any().downcast_ref::() { if let Some(result) = analyze_window_sort_removal( exec.window_expr(), &exec.partition_keys, - &sort_input, - sort_output_ordering, - sort_input_ordering, sort_onward, )? { return Ok(Some(result)); @@ -432,9 +425,6 @@ fn ensure_sorting( if let Some(result) = analyze_window_sort_removal( exec.window_expr(), &exec.partition_keys, - &sort_input, - sort_output_ordering, - sort_input_ordering, sort_onward, )? { return Ok(Some(result)); @@ -534,26 +524,48 @@ fn analyze_immediate_sort_removal( fn analyze_window_sort_removal( window_expr: &[Arc], partition_keys: &[Arc], - sort_input: &Arc, - sort_output_ordering: Option<&[PhysicalSortExpr]>, - physical_ordering: Option<&[PhysicalSortExpr]>, sort_onward: &mut ExecTree, ) -> Result> { - let required_ordering = sort_output_ordering.ok_or_else(|| { - DataFusionError::Plan("A SortExec should have output ordering".to_string()) - })?; - let physical_ordering = if let Some(physical_ordering) = physical_ordering { - physical_ordering - } else { - // If there is no physical ordering, there is no way to remove a sort -- immediately return: - return Ok(None); - }; - let (can_skip_sorting, should_reverse) = can_skip_sort( - window_expr[0].partition_by(), - required_ordering, - &sort_input.schema(), - physical_ordering, - )?; + let bottom_sorts = sort_onward.bottom_executors(); + let mut can_skip_sortings = vec![]; + let mut should_reverses = vec![]; + let mut physical_ordering_common = vec![]; + for sort_any in bottom_sorts { + let sort_output_ordering = sort_any.output_ordering(); + // Variable `sort_any` will either be a `SortExec` or a + // `SortPreservingMergeExec`, and both have single child. + // Therefore, we can use the 0th index without loss of generality. + let sort_input = sort_any.children()[0].clone(); + let physical_ordering = sort_input.output_ordering(); + let required_ordering = sort_output_ordering.ok_or_else(|| { + DataFusionError::Plan("A SortExec should have output ordering".to_string()) + })?; + let physical_ordering = if let Some(physical_ordering) = physical_ordering { + physical_ordering + } else { + // If there is no physical ordering, there is no way to remove a sort -- immediately return: + return Ok(None); + }; + if physical_ordering.len() < physical_ordering_common.len() + || physical_ordering_common.is_empty() + { + physical_ordering_common = physical_ordering.to_vec(); + } + let (can_skip_sorting, should_reverse) = can_skip_sort( + window_expr[0].partition_by(), + required_ordering, + &sort_input.schema(), + physical_ordering, + )?; + can_skip_sortings.push(can_skip_sorting); + should_reverses.push(should_reverse); + } + let can_skip_sorting = can_skip_sortings.iter().all(|elem| *elem); + let can_skip_sorting = can_skip_sorting + && should_reverses + .iter() + .all(|elem| *elem == should_reverses[0]); + let should_reverse = should_reverses[0]; if can_skip_sorting { let new_window_expr = if should_reverse { window_expr @@ -575,7 +587,7 @@ fn analyze_window_sort_removal( new_child, new_schema, partition_keys.to_vec(), - Some(physical_ordering.to_vec()), + Some(physical_ordering_common), )?) as _ } else { Arc::new(WindowAggExec::try_new( @@ -583,7 +595,7 @@ fn analyze_window_sort_removal( new_child, new_schema, partition_keys.to_vec(), - Some(physical_ordering.to_vec()), + Some(physical_ordering_common), )?) as _ }; return Ok(Some(PlanWithCorrespondingSort::new(new_plan))); diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 9170d7ce65b7a..248835c6b2f87 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2419,6 +2419,52 @@ async fn test_window_agg_global_sort() -> Result<()> { Ok(()) } +#[tokio::test] +async fn test_window_agg_with_global_limit() -> Result<()> { + let config = SessionConfig::new() + .with_repartition_windows(false) + .with_target_partitions(1); + let ctx = SessionContext::with_config(config); + register_aggregate_csv(&ctx).await?; + let sql = "SELECT ARRAY_AGG(c13) as array_agg1 FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 1)"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + let formatted = displayable(physical_plan.as_ref()).indent().to_string(); + // Only 1 SortExec was added + let expected = { + vec![ + "ProjectionExec: expr=[ARRAYAGG(aggregate_test_100.c13)@0 as array_agg1]", + " AggregateExec: mode=Final, gby=[], aggr=[ARRAYAGG(aggregate_test_100.c13)]", + " AggregateExec: mode=Partial, gby=[], aggr=[ARRAYAGG(aggregate_test_100.c13)]", + " GlobalLimitExec: skip=0, fetch=1", + " SortExec: [c13@0 ASC NULLS LAST]", + " ProjectionExec: expr=[c13@0 as c13]", + ] + }; + + let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual_len = actual.len(); + let actual_trim_last = &actual[..actual_len - 1]; + assert_eq!( + expected, actual_trim_last, + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let actual = execute_to_batches(&ctx, sql).await; + let expected = vec![ + "+----------------------------------+", + "| array_agg1 |", + "+----------------------------------+", + "| [0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm] |", + "+----------------------------------+", + ]; + assert_batches_eq!(expected, &actual); + + Ok(()) +} + fn write_test_data_to_parquet(tmpdir: &TempDir, n_file: usize) -> Result<()> { let ts_field = Field::new("ts", DataType::Int32, false); let inc_field = Field::new("inc_col", DataType::Int32, false); From 4d8a4b27268b357fe4b0c623eac0cbfdfe3d981e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 31 Jan 2023 11:36:57 +0300 Subject: [PATCH 05/31] minor changes --- .../physical_optimizer/sort_enforcement.rs | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index e2e43afa23961..cb6f88f9125f1 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -97,9 +97,9 @@ impl ExecTree { #[derive(Debug, Clone)] struct PlanWithCorrespondingSort { plan: Arc, - // For every child, keep a vector of `ExecutionPlan`s starting from the - // closest `SortExec` till the current plan. The first index of the tuple is - // the child index of the plan -- we need this information as we make updates. + // For every child, keep a tree of `ExecutionPlan`s starting from the + // child till SortExecs that determine ordering of child. If child has linkage to nay Sort + // do not keep a tree for that child (e.g use None). sort_onwards: Vec>, } @@ -140,40 +140,40 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { .map(|(idx, item)| { let flags = item.plan.maintains_input_order(); let required_ordering = item.plan.required_input_ordering(); - // The `sort_onwards` list starts from the sort-introducing operator - // (e.g `SortExec`, `SortPreservingMergeExec`) and collects all the - // intermediate executors that maintain this ordering. If we are at - // the beginning, both `SortExec` and `SortPreservingMergeExec` doesn't - // maintain ordering (as they introduce the ordering). However, we want - // to propagate them above anyway. let mut res = vec![]; for (maintains, element, required_ordering) in izip!(flags, item.sort_onwards.clone(), required_ordering) { - if (maintains || item.plan.as_any().is::()) + if (maintains + // partially maintains ordering + || (item.plan.as_any().is::() && item.plan.output_ordering().is_some())) && element.is_some() && required_ordering.is_none() // Sorts under limit are not removable. && !is_limit(&item.plan) { - // return element.clone(); res.push(element.clone().unwrap()); } } - // None - if !res.is_empty() { + // The `sort_onwards` list starts from the sort-introducing operator + // (e.g `SortExec`, `SortPreservingMergeExec`) and collects all the + // intermediate executors that maintain this ordering. If we are at + // the beginning, we reset tree, and start from bottom. + if is_sort(&item.plan) { Some(ExecTree { idx, plan: item.plan.clone(), - children: res, + children: vec![], }) - } else if is_sort(&item.plan) { + } // Add parent node to the tree + else if !res.is_empty() { Some(ExecTree { idx, plan: item.plan.clone(), - children: vec![], + children: res, }) } else { + // There is no sort linkage for this child None } }) From 3884fd5feecd07170404d72b533b50de2f5d1460 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 31 Jan 2023 14:43:46 +0300 Subject: [PATCH 06/31] Coalesce Partitions converted to tree --- .../physical_optimizer/sort_enforcement.rs | 178 +++++++++++++----- 1 file changed, 127 insertions(+), 51 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index cb6f88f9125f1..d9f8f70736ca3 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -197,14 +197,15 @@ struct PlanWithCorrespondingCoalescePartitions { // `CoalescePartitionsExec` till the current plan. Since we are sure // that a `CoalescePartitionsExec` can only be propagated on executors // with a single child, we use a single vector (not one for each child). - coalesce_onwards: Vec>, + coalesce_onwards: Vec>, } impl PlanWithCorrespondingCoalescePartitions { pub fn new(plan: Arc) -> Self { + let length = plan.children().len(); PlanWithCorrespondingCoalescePartitions { plan, - coalesce_onwards: vec![], + coalesce_onwards: vec![None; length], } } @@ -230,11 +231,39 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { .into_iter() .map(transform) .collect::>>()?; - let coalesce_onwards = children_requirements - .iter() - .map(|item| item.coalesce_onwards.clone()) - .next() - .unwrap(); + let mut coalesce_onwards = vec![]; + for (idx, item) in children_requirements.iter().enumerate() { + if item.plan.as_any().is::() { + coalesce_onwards.push(Some(ExecTree { + idx, + children: vec![], + plan: item.plan.clone(), + })) + } else if !item.plan.children().is_empty() { + let requires_single_partition = matches!( + item.plan.required_input_distribution()[0], + Distribution::SinglePartition + ); + // The Executor above(not necessarily immediately above) CoalescePartitionExec doesn't require SinglePartition + // hence CoalescePartitionExec is not a requirement of this executor. + // We can propagate it upwards. + let mut res = vec![]; + for elem in item.coalesce_onwards.iter().flatten() { + res.push(elem.clone()); + } + if !requires_single_partition && !res.is_empty() { + coalesce_onwards.push(Some(ExecTree { + idx, + children: res, + plan: item.plan.clone(), + })); + } else { + coalesce_onwards.push(None); + } + } else { + coalesce_onwards.push(None); + } + } let children_plans = children_requirements .iter() .map(|item| item.plan.clone()) @@ -294,10 +323,6 @@ fn parallelize_sorts( if plan.children().is_empty() { return Ok(None); } - let requires_single_partition = matches!( - plan.required_input_distribution()[0], - Distribution::SinglePartition - ); let mut coalesce_onwards = requirements.coalesce_onwards; if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_expr = sort_exec.expr(); @@ -305,51 +330,33 @@ fn parallelize_sorts( // (e.g all Executors in between doesn't require to work on SinglePartition). // We can replace CoalescePartitionsExec with SortExec and SortExec with SortPreservingMergeExec // respectively to parallelize sorting. - if let [first, ..] = coalesce_onwards.as_slice() { - let mut prev_layer = first.children()[0].clone(); - if !ordering_satisfy(prev_layer.output_ordering(), Some(sort_expr), || { - sort_exec.equivalence_properties() - }) { - prev_layer = add_sort_above_child(&prev_layer, sort_expr.to_vec())?; - }; - // In the loop below, we start from one as the first entry is a - // `CoalescePartitionsExec` and we are removing it from the plan. - for layer in coalesce_onwards.iter().skip(1) { - let mut children = layer.children(); - children[0] = prev_layer; - prev_layer = layer.clone().with_new_children(children)?; - } + if coalesce_onwards[0].is_some() { + let mut prev_layer = plan.clone(); + update_child_to_change_coalesce( + &mut prev_layer, + &mut coalesce_onwards[0], + sort_exec, + )?; + let spm = SortPreservingMergeExec::new(sort_expr.to_vec(), prev_layer); return Ok(Some(PlanWithCorrespondingCoalescePartitions { plan: Arc::new(spm), - coalesce_onwards: vec![], - })); - } - } else if plan.as_any().is::() { - // There is a CoalescePartitions that is unnecesary, remove it. - if let [first, ..] = coalesce_onwards.as_slice() { - let mut prev_layer = first.children()[0].clone(); - for layer in coalesce_onwards.iter().skip(1) { - let mut children = layer.children(); - children[0] = prev_layer; - prev_layer = layer.clone().with_new_children(children)?; - } - let new_plan = plan.clone().with_new_children(vec![prev_layer])?; - return Ok(Some(PlanWithCorrespondingCoalescePartitions { - plan: new_plan.clone(), - coalesce_onwards: vec![new_plan], + coalesce_onwards: vec![None], })); - } else { - // starting of coalesce partition - coalesce_onwards = vec![plan.clone()]; } - } else if !requires_single_partition && !coalesce_onwards.is_empty() { - // The Executor above(not necessarily immediately above) CoalescePartitionExec doesn't require SinglePartition - // hence CoalescePartitionExec is not a requirement of this executor. - // We can propagate it upwards. - coalesce_onwards.push(plan.clone()); - } else { - coalesce_onwards.clear(); + } else if plan.as_any().is::() + && coalesce_onwards[0].is_some() + { + let mut prev_layer = plan.clone(); + update_child_to_remove_unnecessary_coalesce( + &mut prev_layer, + &mut coalesce_onwards[0], + )?; + let new_plan = plan.with_new_children(vec![prev_layer])?; + return Ok(Some(PlanWithCorrespondingCoalescePartitions { + plan: new_plan, + coalesce_onwards: vec![None], + })); } Ok(Some(PlanWithCorrespondingCoalescePartitions { @@ -604,6 +611,29 @@ fn analyze_window_sort_removal( Ok(None) } +/// Updates child to remove the unnecessary sorting below it. +fn update_child_to_change_coalesce( + child: &mut Arc, + sort_onwards: &mut Option, + sort_exec: &SortExec, +) -> Result<()> { + if let Some(sort_onwards) = sort_onwards { + *child = change_corresponding_coalesce_in_sub_plan(sort_onwards, sort_exec)?; + } + Ok(()) +} + +/// Updates child to remove the unnecessary sorting below it. +fn update_child_to_remove_unnecessary_coalesce( + child: &mut Arc, + sort_onwards: &mut Option, +) -> Result<()> { + if let Some(sort_onwards) = sort_onwards { + *child = remove_corresponding_coalesce_in_sub_plan(sort_onwards)?; + } + Ok(()) +} + /// Updates child to remove the unnecessary sorting below it. fn update_child_to_remove_unnecessary_sort( child: &mut Arc, @@ -643,6 +673,52 @@ fn get_sort_exprs(sort_any: &Arc) -> Result<&[PhysicalSortExp } } +/// Removes the sort from the plan in `sort_onwards`. +fn change_corresponding_coalesce_in_sub_plan( + sort_onwards: &mut ExecTree, + sort_exec: &SortExec, +) -> Result> { + let res = if sort_onwards.plan.as_any().is::() { + let coalesce_input = sort_onwards.plan.children()[0].clone(); + let sort_expr = sort_exec.expr(); + if !ordering_satisfy(coalesce_input.output_ordering(), Some(sort_expr), || { + sort_exec.equivalence_properties() + }) { + add_sort_above_child(&coalesce_input, sort_expr.to_vec())? + } else { + coalesce_input + } + } else { + let res = sort_onwards.plan.clone(); + let mut children = res.children(); + for elem in &mut sort_onwards.children { + children[elem.idx] = + change_corresponding_coalesce_in_sub_plan(elem, sort_exec)?; + } + res.with_new_children(children)? + }; + + Ok(res) +} + +/// Removes the sort from the plan in `sort_onwards`. +fn remove_corresponding_coalesce_in_sub_plan( + sort_onwards: &mut ExecTree, +) -> Result> { + let res = if sort_onwards.plan.as_any().is::() { + sort_onwards.plan.children()[0].clone() + } else { + let res = sort_onwards.plan.clone(); + let mut children = res.children(); + for elem in &mut sort_onwards.children { + children[elem.idx] = remove_corresponding_coalesce_in_sub_plan(elem)?; + } + res.with_new_children(children)? + }; + + Ok(res) +} + /// Removes the sort from the plan in `sort_onwards`. fn remove_corresponding_sort_from_sub_plan( sort_onwards: &mut ExecTree, From 32ff1b0696f424974fcb7564d78c3ef7a203882f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 31 Jan 2023 15:08:34 +0300 Subject: [PATCH 07/31] Simplifications --- .../physical_optimizer/sort_enforcement.rs | 182 ++++++++---------- 1 file changed, 79 insertions(+), 103 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index d9f8f70736ca3..8dd54a8a5e938 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -233,36 +233,36 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { .collect::>>()?; let mut coalesce_onwards = vec![]; for (idx, item) in children_requirements.iter().enumerate() { + let mut coalesce_onward = None; + // coalesce_onwards starts from CoalescePartitionsExec. Hence reset child when + // CoalescePartitionsExec is encountered if item.plan.as_any().is::() { - coalesce_onwards.push(Some(ExecTree { + coalesce_onward = Some(ExecTree { idx, children: vec![], plan: item.plan.clone(), - })) + }); } else if !item.plan.children().is_empty() { let requires_single_partition = matches!( item.plan.required_input_distribution()[0], Distribution::SinglePartition ); - // The Executor above(not necessarily immediately above) CoalescePartitionExec doesn't require SinglePartition - // hence CoalescePartitionExec is not a requirement of this executor. - // We can propagate it upwards. let mut res = vec![]; for elem in item.coalesce_onwards.iter().flatten() { res.push(elem.clone()); } + // The Executor above(not necessarily immediately above) CoalescePartitionExec doesn't require SinglePartition + // hence CoalescePartitionExec is not a requirement of this executor. + // We can propagate it upwards. if !requires_single_partition && !res.is_empty() { - coalesce_onwards.push(Some(ExecTree { + coalesce_onward = Some(ExecTree { idx, children: res, plan: item.plan.clone(), - })); - } else { - coalesce_onwards.push(None); + }); } - } else { - coalesce_onwards.push(None); } + coalesce_onwards.push(coalesce_onward); } let children_plans = children_requirements .iter() @@ -335,7 +335,7 @@ fn parallelize_sorts( update_child_to_change_coalesce( &mut prev_layer, &mut coalesce_onwards[0], - sort_exec, + Some(sort_exec), )?; let spm = SortPreservingMergeExec::new(sort_expr.to_vec(), prev_layer); @@ -347,11 +347,9 @@ fn parallelize_sorts( } else if plan.as_any().is::() && coalesce_onwards[0].is_some() { + // There is a redundant CoalescePartitionExec in the plan let mut prev_layer = plan.clone(); - update_child_to_remove_unnecessary_coalesce( - &mut prev_layer, - &mut coalesce_onwards[0], - )?; + update_child_to_change_coalesce(&mut prev_layer, &mut coalesce_onwards[0], None)?; let new_plan = plan.with_new_children(vec![prev_layer])?; return Ok(Some(PlanWithCorrespondingCoalescePartitions { plan: new_plan, @@ -506,7 +504,6 @@ fn analyze_immediate_sort_removal( let new_plan: Arc = Arc::new( SortPreservingMergeExec::new(sort_exec.expr().to_vec(), sort_input), ); - // new_onwards.push((0, new_plan.clone())); new_onwards = Some(ExecTree { idx: 0, plan: new_plan.clone(), @@ -611,87 +608,45 @@ fn analyze_window_sort_removal( Ok(None) } -/// Updates child to remove the unnecessary sorting below it. +/// Updates child to remove the unnecessary coalesce partitions below it. fn update_child_to_change_coalesce( child: &mut Arc, - sort_onwards: &mut Option, - sort_exec: &SortExec, -) -> Result<()> { - if let Some(sort_onwards) = sort_onwards { - *child = change_corresponding_coalesce_in_sub_plan(sort_onwards, sort_exec)?; - } - Ok(()) -} - -/// Updates child to remove the unnecessary sorting below it. -fn update_child_to_remove_unnecessary_coalesce( - child: &mut Arc, - sort_onwards: &mut Option, + coalesce_onwards: &mut Option, + sort_exec: Option<&SortExec>, ) -> Result<()> { - if let Some(sort_onwards) = sort_onwards { - *child = remove_corresponding_coalesce_in_sub_plan(sort_onwards)?; + if let Some(coalesce_onwards) = coalesce_onwards { + *child = change_corresponding_coalesce_in_sub_plan(coalesce_onwards, sort_exec)?; } Ok(()) } -/// Updates child to remove the unnecessary sorting below it. -fn update_child_to_remove_unnecessary_sort( - child: &mut Arc, - sort_onwards: &mut Option, -) -> Result<()> { - if let Some(sort_onwards) = sort_onwards { - *child = remove_corresponding_sort_from_sub_plan(sort_onwards)?; - } - Ok(()) -} - -/// Updates child to modify the unnecessarily fine sorting below it. -fn update_child_to_change_finer_sort( - child: &mut Arc, - sort_onwards: &mut Option, - n_sort_expr: usize, -) -> Result<()> { - if let Some(sort_onwards) = sort_onwards { - *child = change_finer_sort_in_sub_plan(sort_onwards, n_sort_expr)?; - } - Ok(()) -} - -/// Converts an [ExecutionPlan] trait object to a [PhysicalSortExpr] slice when possible. -fn get_sort_exprs(sort_any: &Arc) -> Result<&[PhysicalSortExpr]> { - if let Some(sort_exec) = sort_any.as_any().downcast_ref::() { - Ok(sort_exec.expr()) - } else if let Some(sort_preserving_merge_exec) = - sort_any.as_any().downcast_ref::() - { - Ok(sort_preserving_merge_exec.expr()) - } else { - Err(DataFusionError::Plan( - "Given ExecutionPlan is not a SortExec or a SortPreservingMergeExec" - .to_string(), - )) - } -} - -/// Removes the sort from the plan in `sort_onwards`. +/// Removes the coalesce from the plan in `coalesce_onwards`. fn change_corresponding_coalesce_in_sub_plan( - sort_onwards: &mut ExecTree, - sort_exec: &SortExec, + coalesce_onwards: &mut ExecTree, + sort_exec: Option<&SortExec>, ) -> Result> { - let res = if sort_onwards.plan.as_any().is::() { - let coalesce_input = sort_onwards.plan.children()[0].clone(); - let sort_expr = sort_exec.expr(); - if !ordering_satisfy(coalesce_input.output_ordering(), Some(sort_expr), || { - sort_exec.equivalence_properties() - }) { - add_sort_above_child(&coalesce_input, sort_expr.to_vec())? - } else { - coalesce_input + let res = if coalesce_onwards + .plan + .as_any() + .is::() + { + let mut coalesce_input = coalesce_onwards.plan.children()[0].clone(); + if let Some(sort_exec) = sort_exec { + let sort_expr = sort_exec.expr(); + if !ordering_satisfy( + coalesce_input.output_ordering(), + Some(sort_expr), + || sort_exec.equivalence_properties(), + ) { + coalesce_input = + add_sort_above_child(&coalesce_input, sort_expr.to_vec())?; + } } + coalesce_input } else { - let res = sort_onwards.plan.clone(); + let res = coalesce_onwards.plan.clone(); let mut children = res.children(); - for elem in &mut sort_onwards.children { + for elem in &mut coalesce_onwards.children { children[elem.idx] = change_corresponding_coalesce_in_sub_plan(elem, sort_exec)?; } @@ -701,22 +656,15 @@ fn change_corresponding_coalesce_in_sub_plan( Ok(res) } -/// Removes the sort from the plan in `sort_onwards`. -fn remove_corresponding_coalesce_in_sub_plan( - sort_onwards: &mut ExecTree, -) -> Result> { - let res = if sort_onwards.plan.as_any().is::() { - sort_onwards.plan.children()[0].clone() - } else { - let res = sort_onwards.plan.clone(); - let mut children = res.children(); - for elem in &mut sort_onwards.children { - children[elem.idx] = remove_corresponding_coalesce_in_sub_plan(elem)?; - } - res.with_new_children(children)? - }; - - Ok(res) +/// Updates child to remove the unnecessary sorting below it. +fn update_child_to_remove_unnecessary_sort( + child: &mut Arc, + sort_onwards: &mut Option, +) -> Result<()> { + if let Some(sort_onwards) = sort_onwards { + *child = remove_corresponding_sort_from_sub_plan(sort_onwards)?; + } + Ok(()) } /// Removes the sort from the plan in `sort_onwards`. @@ -737,6 +685,18 @@ fn remove_corresponding_sort_from_sub_plan( Ok(res) } +/// Updates child to modify the unnecessarily fine sorting below it. +fn update_child_to_change_finer_sort( + child: &mut Arc, + sort_onwards: &mut Option, + n_sort_expr: usize, +) -> Result<()> { + if let Some(sort_onwards) = sort_onwards { + *child = change_finer_sort_in_sub_plan(sort_onwards, n_sort_expr)?; + } + Ok(()) +} + /// Change the unnecessarily fine sort in `sort_onwards`. fn change_finer_sort_in_sub_plan( sort_onwards: &mut ExecTree, @@ -750,7 +710,7 @@ fn change_finer_sort_in_sub_plan( let res = sort_onwards.plan.clone(); let mut children = res.children(); for elem in &mut sort_onwards.children { - children[elem.idx] = remove_corresponding_sort_from_sub_plan(elem)?; + children[elem.idx] = change_finer_sort_in_sub_plan(elem, n_sort_expr)?; } res.with_new_children(children)? }; @@ -758,6 +718,22 @@ fn change_finer_sort_in_sub_plan( Ok(res) } +/// Converts an [ExecutionPlan] trait object to a [PhysicalSortExpr] slice when possible. +fn get_sort_exprs(sort_any: &Arc) -> Result<&[PhysicalSortExpr]> { + if let Some(sort_exec) = sort_any.as_any().downcast_ref::() { + Ok(sort_exec.expr()) + } else if let Some(sort_preserving_merge_exec) = + sort_any.as_any().downcast_ref::() + { + Ok(sort_preserving_merge_exec.expr()) + } else { + Err(DataFusionError::Plan( + "Given ExecutionPlan is not a SortExec or a SortPreservingMergeExec" + .to_string(), + )) + } +} + #[derive(Debug)] /// This structure stores extra column information required to remove unnecessary sorts. pub struct ColumnInfo { From 2d9819a7e43111b8088ab244a36e2a2875f67668 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 31 Jan 2023 15:36:49 +0300 Subject: [PATCH 08/31] Add new test --- .../physical_optimizer/sort_enforcement.rs | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 8dd54a8a5e938..2a2d7ebc9a932 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -1320,6 +1320,47 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_multilayer_coalesce_partitions() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let repartition = Arc::new(RepartitionExec::try_new( + source1, + Partitioning::RoundRobinBatch(2), + )?) as Arc; + let coalesce = Arc::new(CoalescePartitionsExec::new(repartition)) as _; + // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before + let filter = filter_exec( + Arc::new(NotExpr::new( + col("non_nullable_col", schema.as_ref()).unwrap(), + )), + coalesce, + ); + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let physical_plan = sort_exec(sort_exprs, filter); + + // CoalescePartitionsExec and SortExec are not directly consecutive. In this case + // we should be able to parallelize Sorting also (given that executors in between don't require) + // single partition. + let expected_input = vec![ + "SortExec: [nullable_col@0 ASC]", + " FilterExec: NOT non_nullable_col@1", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " FilterExec: NOT non_nullable_col@1", + " SortExec: [nullable_col@0 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] // With new change in SortEnforcement EnforceSorting->EnforceDistribution->EnforceSorting // should produce same result with EnforceDistribution+EnforceSorting From dff1f7fcd28ed3c70f47ecf97c46cef2cec4c3ea Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 31 Jan 2023 15:48:19 +0300 Subject: [PATCH 09/31] Simplifications --- .../src/physical_optimizer/sort_enforcement.rs | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 2a2d7ebc9a932..2b0e1c8bfe848 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -345,6 +345,7 @@ fn parallelize_sorts( })); } } else if plan.as_any().is::() + // CoalescePartitionsExec have single child, we can use 0th index safely. && coalesce_onwards[0].is_some() { // There is a redundant CoalescePartitionExec in the plan @@ -378,6 +379,9 @@ fn ensure_sorting( requirements: PlanWithCorrespondingSort, ) -> Result> { // Perform naive analysis at the beginning -- remove already-satisfied sorts: + if requirements.plan.children().is_empty() { + return Ok(None); + } if let Some(result) = analyze_immediate_sort_removal(&requirements)? { return Ok(Some(result)); } @@ -466,15 +470,11 @@ fn ensure_sorting( (None, None) => {} } } - if plan.children().is_empty() { - Ok(Some(requirements)) - } else { - let new_plan = requirements.plan.with_new_children(new_children)?; - Ok(Some(PlanWithCorrespondingSort { - plan: new_plan, - sort_onwards: new_onwards, - })) - } + let new_plan = requirements.plan.with_new_children(new_children)?; + Ok(Some(PlanWithCorrespondingSort { + plan: new_plan, + sort_onwards: new_onwards, + })) } /// Analyzes a given `SortExec` to determine whether its input already has From 17e07eb150a177725732680b7d83e85e89cab464 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 1 Feb 2023 00:15:14 -0600 Subject: [PATCH 10/31] Simplifications and refactors --- datafusion/core/src/execution/context.rs | 7 +- .../physical_optimizer/sort_enforcement.rs | 598 +++++++++--------- 2 files changed, 315 insertions(+), 290 deletions(-) diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 76e7964cef325..92c0fea01f3ad 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1486,10 +1486,9 @@ impl SessionState { // The EnforceSorting rule is for adding essential local sorting to satisfy the required // ordering. Please make sure that the whole plan tree is determined before this rule. // Note that one should always run this rule after running the EnforceDistribution rule - // as the latter may break local sorting requirements. The EnforceSorting rule takes a - // boolean flag indicating whether we elect to transform CoalescePartitionsExec + SortExec - // cascades into SortExec + SortPreservingMergeExec cascades, which enables us to perform - // sorting parallel. + // as the latter may break local sorting requirements. The rule takes a boolean flag + // indicating whether we elect to transform CoalescePartitionsExec + SortExec cascades + // into SortExec + SortPreservingMergeExec cascades, enabling parallel sorting. Arc::new(EnforceSorting::new(true)), // The CoalesceBatches rule will not influence the distribution and ordering of the // whole plan tree. Therefore, to avoid influencing other rules, it should run last. diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 2b0e1c8bfe848..41c9a015c7d15 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -46,7 +46,7 @@ use datafusion_common::{reverse_sort_options, DataFusionError}; use datafusion_physical_expr::utils::{ordering_satisfy, ordering_satisfy_concrete}; use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use itertools::izip; +use itertools::{concat, izip}; use std::iter::zip; use std::sync::Arc; @@ -54,7 +54,7 @@ use std::sync::Arc; /// ones it can prove unnecessary. The boolean flag `parallelize_sorts` /// indicates whether we elect to transform CoalescePartitionsExec + SortExec /// cascades into SortExec + SortPreservingMergeExec cascades, which enables -/// us to perform sorting parallel. +/// us to perform sorting in parallel. #[derive(Default)] pub struct EnforceSorting { parallelize_sorts: bool, @@ -67,39 +67,38 @@ impl EnforceSorting { } } +/// This object implements a tree that we use while keeping track of paths +/// leading to `SortExec`s. #[derive(Debug, Clone)] struct ExecTree { - /// child index of the plan in its parent executor + /// Child index of the plan in its parent pub idx: usize, - /// Children of the `plan` that needs to be updated if bottom executor is removed + /// Children of the plan that would need updating if we remove leaf executors pub children: Vec, - /// ExecutionPlan node of the tree + /// The `ExecutionPlan` associated with this node pub plan: Arc, } impl ExecTree { - /// Executor at the leaf of the tree - fn bottom_executors(&self) -> Vec> { - if !self.children.is_empty() { - let mut res = vec![]; - for child in &self.children { - res.extend(child.bottom_executors()) - } - res - } else { + /// This function returns the executors at the leaves of the tree. + fn get_leaves(&self) -> Vec> { + if self.children.is_empty() { vec![self.plan.clone()] + } else { + concat(self.children.iter().map(|e| e.get_leaves())) } } } -/// This is a "data class" we use within the [EnforceSorting] rule that -/// tracks the closest `SortExec` descendant for every child of a plan. +/// This object is used within the [EnforceSorting] rule to track the closest +/// `SortExec` descendant(s) for every child of a plan. #[derive(Debug, Clone)] struct PlanWithCorrespondingSort { plan: Arc, - // For every child, keep a tree of `ExecutionPlan`s starting from the - // child till SortExecs that determine ordering of child. If child has linkage to nay Sort - // do not keep a tree for that child (e.g use None). + // For every child, keep a subtree of `ExecutionPlan`s starting from the + // child until the `SortExec`(s) -- could be multiple for n-ary plans like + // Union -- that determine the output ordering of the child. If the child + // has no connection to any sort, simpliy store None (and not a subtree). sort_onwards: Vec>, } @@ -134,69 +133,80 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { .into_iter() .map(transform) .collect::>>()?; - let sort_onwards = children_requirements + let children_plans = children_requirements .iter() + .map(|item| item.plan.clone()) + .collect(); + let sort_onwards = children_requirements + .into_iter() .enumerate() .map(|(idx, item)| { - let flags = item.plan.maintains_input_order(); - let required_ordering = item.plan.required_input_ordering(); - let mut res = vec![]; - for (maintains, element, required_ordering) in - izip!(flags, item.sort_onwards.clone(), required_ordering) - { - if (maintains - // partially maintains ordering - || (item.plan.as_any().is::() && item.plan.output_ordering().is_some())) - && element.is_some() - && required_ordering.is_none() - // Sorts under limit are not removable. - && !is_limit(&item.plan) - { - res.push(element.clone().unwrap()); - } - } - // The `sort_onwards` list starts from the sort-introducing operator - // (e.g `SortExec`, `SortPreservingMergeExec`) and collects all the - // intermediate executors that maintain this ordering. If we are at - // the beginning, we reset tree, and start from bottom. + // Leaves of the `sort_onwards` tree are sort-introducing operators + // (e.g `SortExec`, `SortPreservingMergeExec`). This tree collects + // all the intermediate executors that maintain this ordering. If + // we just saw a sort-introducing operator, we reset the tree and + // start accumulating. if is_sort(&item.plan) { - Some(ExecTree { + return Some(ExecTree { idx, - plan: item.plan.clone(), + plan: item.plan, children: vec![], + }); + // TODO: I moved this check here, I think you can exit early like this. + // Please double check. + } else if is_limit(&item.plan) { + // There is no sort linkage for this path, it starts at a limit. + return None; + } + let plan = &item.plan; + let union = plan.as_any().is::(); + let flags = plan.maintains_input_order(); + let required_orderings = plan.required_input_ordering(); + let children = izip!(flags, item.sort_onwards, required_orderings) + .filter_map(|(maintains, element, required_ordering)| { + // TODO: Why would we check if the plan is a Union every time in the closure? + // I moved it outside. Also, unless I am missing something, if the plan + // is a union, it maintains ordering (for any input) *if and only if* it + // has some output ordering. Assuming that this is indeed true, I removed + // the "plan.output_ordering.is_some()" predicate. Please double check if + // this is correct. + if required_ordering.is_none() && (maintains || union) { + element + } else { + None + } }) - } // Add parent node to the tree - else if !res.is_empty() { + .collect::>(); + if !children.is_empty() { + // Add parent node to the tree if there is at least one + // child with a subtree: Some(ExecTree { idx, - plan: item.plan.clone(), - children: res, + plan: item.plan, + children, }) } else { - // There is no sort linkage for this child + // There is no sort linkage for this child, do nothing. None } }) - .collect::>(); - let children_plans = children_requirements - .iter() - .map(|item| item.plan.clone()) - .collect::>(); + .collect(); let plan = with_new_children_if_necessary(self.plan, children_plans)?; Ok(PlanWithCorrespondingSort { plan, sort_onwards }) } } } -/// This is a "data class" we use within the [EnforceSorting] rule that -/// tracks the closest `CoalescePartitionsExec` descendant of a plan. +/// This object is used within the [EnforceSorting] rule to track the closest +/// `CoalescePartitionsExec` descendant(s) for every child of a plan. #[derive(Debug, Clone)] struct PlanWithCorrespondingCoalescePartitions { plan: Arc, - // Keep a vector of `ExecutionPlan`s starting from the closest - // `CoalescePartitionsExec` till the current plan. Since we are sure - // that a `CoalescePartitionsExec` can only be propagated on executors - // with a single child, we use a single vector (not one for each child). + // For every child, keep a subtree of `ExecutionPlan`s starting from the + // child until the `CoalescePartitionsExec`(s) -- could be multiple for + // n-ary plans like Union -- that affect the output partitioning of the + // child. If the child has no connection to any `CoalescePartitionsExec`, + // simpliy store None (and not a subtree). coalesce_onwards: Vec>, } @@ -231,43 +241,54 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { .into_iter() .map(transform) .collect::>>()?; - let mut coalesce_onwards = vec![]; - for (idx, item) in children_requirements.iter().enumerate() { - let mut coalesce_onward = None; - // coalesce_onwards starts from CoalescePartitionsExec. Hence reset child when - // CoalescePartitionsExec is encountered - if item.plan.as_any().is::() { - coalesce_onward = Some(ExecTree { - idx, - children: vec![], - plan: item.plan.clone(), - }); - } else if !item.plan.children().is_empty() { - let requires_single_partition = matches!( - item.plan.required_input_distribution()[0], - Distribution::SinglePartition - ); - let mut res = vec![]; - for elem in item.coalesce_onwards.iter().flatten() { - res.push(elem.clone()); - } - // The Executor above(not necessarily immediately above) CoalescePartitionExec doesn't require SinglePartition - // hence CoalescePartitionExec is not a requirement of this executor. - // We can propagate it upwards. - if !requires_single_partition && !res.is_empty() { - coalesce_onward = Some(ExecTree { - idx, - children: res, - plan: item.plan.clone(), - }); - } - } - coalesce_onwards.push(coalesce_onward); - } let children_plans = children_requirements .iter() .map(|item| item.plan.clone()) - .collect::>(); + .collect(); + let coalesce_onwards = children_requirements + .into_iter() + .enumerate() + .map(|(idx, item)| { + // Leaves of the `coalesce_onwards` tree are `CoalescePartitionsExec` + // operators. This tree collects all the intermediate executors that + // maintain a single partition. If we just saw a `CoalescePartitionsExec` + // operator, we reset the tree and start accumulating. + let plan = item.plan; + if plan.as_any().is::() { + Some(ExecTree { + idx, + plan, + children: vec![], + }) + } else if plan.children().is_empty() + || matches!( + // TODO: Please double check if using the first element here is a + // problem or not. Can `plan` not have multiple children? + plan.required_input_distribution()[0], + Distribution::SinglePartition + ) + { + // If the plan has no children, or it doesn't require a + // single partition, there is no linkage. + None + } else { + let children = item + .coalesce_onwards + .into_iter() + .flatten() + .collect::>(); + if children.is_empty() { + None + } else { + Some(ExecTree { + idx, + plan, + children, + }) + } + } + }) + .collect(); let plan = with_new_children_if_necessary(self.plan, children_plans)?; Ok(PlanWithCorrespondingCoalescePartitions { plan, @@ -324,73 +345,77 @@ fn parallelize_sorts( return Ok(None); } let mut coalesce_onwards = requirements.coalesce_onwards; - if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let sort_expr = sort_exec.expr(); - // If there is link between CoalescePartitionsExec and SortExec that satisfy requirements - // (e.g all Executors in between doesn't require to work on SinglePartition). - // We can replace CoalescePartitionsExec with SortExec and SortExec with SortPreservingMergeExec - // respectively to parallelize sorting. - if coalesce_onwards[0].is_some() { + // We know that `plan` has children, so `coalesce_onwards` is non-empty. + if coalesce_onwards[0].is_some() { + if let Some(sort_exec) = plan.as_any().downcast_ref::() { + // If there is a connection between a `CoalescePartitionsExec` and a + // `SortExec` that satisfy the requirements (i.e. they don't require a + // single partition), then we can replace the `CoalescePartitionsExec` + // + `SortExec` cascade with a `SortExec` + `SortPreservingMergeExec` + // cascade to parallelize sorting. let mut prev_layer = plan.clone(); update_child_to_change_coalesce( &mut prev_layer, &mut coalesce_onwards[0], Some(sort_exec), )?; - - let spm = SortPreservingMergeExec::new(sort_expr.to_vec(), prev_layer); + let spm = SortPreservingMergeExec::new(sort_exec.expr().to_vec(), prev_layer); return Ok(Some(PlanWithCorrespondingCoalescePartitions { plan: Arc::new(spm), coalesce_onwards: vec![None], })); + } else if plan.as_any().is::() { + // There is an unnecessary `CoalescePartitionExec` in the plan. + let mut prev_layer = plan.clone(); + update_child_to_change_coalesce( + &mut prev_layer, + &mut coalesce_onwards[0], + None, + )?; + let new_plan = plan.with_new_children(vec![prev_layer])?; + return Ok(Some(PlanWithCorrespondingCoalescePartitions { + plan: new_plan, + coalesce_onwards: vec![None], + })); } - } else if plan.as_any().is::() - // CoalescePartitionsExec have single child, we can use 0th index safely. - && coalesce_onwards[0].is_some() - { - // There is a redundant CoalescePartitionExec in the plan - let mut prev_layer = plan.clone(); - update_child_to_change_coalesce(&mut prev_layer, &mut coalesce_onwards[0], None)?; - let new_plan = plan.with_new_children(vec![prev_layer])?; - return Ok(Some(PlanWithCorrespondingCoalescePartitions { - plan: new_plan, - coalesce_onwards: vec![None], - })); } - Ok(Some(PlanWithCorrespondingCoalescePartitions { plan, coalesce_onwards, })) } -// Checks whether executor is limit executor (e.g either LocalLimitExec or GlobalLimitExec) +/// Checks whether the given executor is a limit; +/// i.e. either a `LocalLimitExec` or a `GlobalLimitExec`. fn is_limit(plan: &Arc) -> bool { plan.as_any().is::() || plan.as_any().is::() } -// Checks whether executor is Sort -// TODO: Add support for SortPreservingMergeExec also. +/// Checks whether the given executor is sort-introducing. fn is_sort(plan: &Arc) -> bool { + // TODO: Add support for SortPreservingMergeExec also. + // This was your comment from before, let's fix it. plan.as_any().is::() } +/// This function enforces sorting requirements and makes optimizations without +/// violating these requirements whenever possible. fn ensure_sorting( requirements: PlanWithCorrespondingSort, ) -> Result> { // Perform naive analysis at the beginning -- remove already-satisfied sorts: - if requirements.plan.children().is_empty() { + let plan = requirements.plan; + let mut children = plan.children(); + if children.is_empty() { return Ok(None); } - if let Some(result) = analyze_immediate_sort_removal(&requirements)? { + let mut sort_onwards = requirements.sort_onwards; + if let Some(result) = analyze_immediate_sort_removal(&plan, &sort_onwards) { return Ok(Some(result)); } - let plan = &requirements.plan; - let mut new_children = plan.children().clone(); - let mut new_onwards = requirements.sort_onwards.clone(); for (idx, (child, sort_onwards, required_ordering)) in izip!( - new_children.iter_mut(), - new_onwards.iter_mut(), + children.iter_mut(), + sort_onwards.iter_mut(), plan.required_input_ordering() ) .enumerate() @@ -408,37 +433,36 @@ fn ensure_sorting( update_child_to_remove_unnecessary_sort(child, sort_onwards)?; let sort_expr = required_ordering.to_vec(); *child = add_sort_above_child(child, sort_expr)?; - // sort_onwards.push((idx, child.clone())) *sort_onwards = Some(ExecTree { idx, plan: child.clone(), children: vec![], }) } - if let Some(sort_onward) = sort_onwards { + if let Some(tree) = sort_onwards { // For window expressions, we can remove some sorts when we can // calculate the result in reverse: if let Some(exec) = plan.as_any().downcast_ref::() { if let Some(result) = analyze_window_sort_removal( exec.window_expr(), &exec.partition_keys, - sort_onward, + tree, )? { return Ok(Some(result)); } - } else if let Some(exec) = requirements - .plan - .as_any() - .downcast_ref::() + } else if let Some(exec) = + plan.as_any().downcast_ref::() { if let Some(result) = analyze_window_sort_removal( exec.window_expr(), &exec.partition_keys, - sort_onward, + tree, )? { return Ok(Some(result)); } } + // TODO: The following is an old note. Is it still true? Please double check + // and we can discuss at our next meeting. // TODO: Once we can ensure that required ordering information propagates with // necessary lineage information, compare `sort_input_ordering` and `required_ordering`. // This will enable us to handle cases such as (a,b) -> Sort -> (a,b,c) -> Required(a,b). @@ -446,9 +470,8 @@ fn ensure_sorting( } } (Some(required), None) => { - // Ordering requirement is not met, we should add a SortExec to the plan. - let sort_expr = required.to_vec(); - *child = add_sort_above_child(child, sort_expr)?; + // Ordering requirement is not met, we should add a `SortExec` to the plan. + *child = add_sort_above_child(child, required.to_vec())?; *sort_onwards = Some(ExecTree { idx, plan: child.clone(), @@ -456,11 +479,11 @@ fn ensure_sorting( }) } (None, Some(_)) => { - // We have a SortExec whose effect may be neutralized by a order-imposing - // operator. In this case, remove this sort: + // We have a `SortExec` whose effect may be neutralized by + // another order-imposing operator. Remove or update this sort: if !plan.maintains_input_order()[idx] { - if plan.output_ordering().is_some() && !is_sort(plan) { - let count = plan.output_ordering().unwrap().len(); + let count = plan.output_ordering().map_or(0, |e| e.len()); + if (count > 0) && !is_sort(&plan) { update_child_to_change_finer_sort(child, sort_onwards, count)?; } else { update_child_to_remove_unnecessary_sort(child, sort_onwards)?; @@ -470,19 +493,19 @@ fn ensure_sorting( (None, None) => {} } } - let new_plan = requirements.plan.with_new_children(new_children)?; Ok(Some(PlanWithCorrespondingSort { - plan: new_plan, - sort_onwards: new_onwards, + plan: plan.with_new_children(children)?, + sort_onwards, })) } -/// Analyzes a given `SortExec` to determine whether its input already has -/// a finer ordering than this `SortExec` enforces. +/// Analyzes a given `SortExec` (`plan`) to determine whether its input already +/// has a finer ordering than this `SortExec` enforces. fn analyze_immediate_sort_removal( - requirements: &PlanWithCorrespondingSort, -) -> Result> { - if let Some(sort_exec) = requirements.plan.as_any().downcast_ref::() { + plan: &Arc, + sort_onwards: &[Option], +) -> 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( @@ -492,35 +515,36 @@ fn analyze_immediate_sort_removal( ) { // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: - let mut new_onwards = requirements.sort_onwards[0].clone(); - let mut children = vec![]; - if let Some(tmp) = &new_onwards { - children.push(tmp.clone()); - } - let updated_plan = if !sort_exec.preserve_partitioning() - && sort_input.output_partitioning().partition_count() > 1 - { - // Replace the sort with a sort-preserving merge: - let new_plan: Arc = Arc::new( - SortPreservingMergeExec::new(sort_exec.expr().to_vec(), sort_input), - ); - new_onwards = Some(ExecTree { - idx: 0, - plan: new_plan.clone(), - children, - }); - new_plan - } else { - // Remove the sort: - sort_input - }; - return Ok(Some(PlanWithCorrespondingSort { - plan: updated_plan, - sort_onwards: vec![new_onwards], - })); + return Some( + if !sort_exec.preserve_partitioning() + && sort_input.output_partitioning().partition_count() > 1 + { + // Replace the sort with a sort-preserving merge: + let new_plan: Arc = + Arc::new(SortPreservingMergeExec::new( + sort_exec.expr().to_vec(), + sort_input, + )); + let new_tree = ExecTree { + idx: 0, + plan: new_plan.clone(), + children: sort_onwards.iter().flat_map(|e| e.clone()).collect(), + }; + PlanWithCorrespondingSort { + plan: new_plan, + sort_onwards: vec![Some(new_tree)], + } + } else { + // Remove the sort: + PlanWithCorrespondingSort { + plan: sort_input, + sort_onwards: sort_onwards.to_vec(), + } + }, + ); } } - Ok(None) + None } /// Analyzes a [WindowAggExec] or a [BoundedWindowAggExec] to determine whether @@ -528,87 +552,92 @@ fn analyze_immediate_sort_removal( fn analyze_window_sort_removal( window_expr: &[Arc], partition_keys: &[Arc], - sort_onward: &mut ExecTree, + sort_tree: &mut ExecTree, ) -> Result> { - let bottom_sorts = sort_onward.bottom_executors(); - let mut can_skip_sortings = vec![]; - let mut should_reverses = vec![]; + let mut first_should_reverse = None; let mut physical_ordering_common = vec![]; - for sort_any in bottom_sorts { + for sort_any in sort_tree.get_leaves() { let sort_output_ordering = sort_any.output_ordering(); // Variable `sort_any` will either be a `SortExec` or a - // `SortPreservingMergeExec`, and both have single child. + // `SortPreservingMergeExec`, and both have a single child. // Therefore, we can use the 0th index without loss of generality. let sort_input = sort_any.children()[0].clone(); let physical_ordering = sort_input.output_ordering(); let required_ordering = sort_output_ordering.ok_or_else(|| { DataFusionError::Plan("A SortExec should have output ordering".to_string()) })?; - let physical_ordering = if let Some(physical_ordering) = physical_ordering { - physical_ordering + if let Some(physical_ordering) = physical_ordering { + // TODO: Do we need a longest common prefix approach here too? + // Please double check this logic is indeed correct. + if physical_ordering_common.is_empty() + || physical_ordering.len() < physical_ordering_common.len() + { + physical_ordering_common = physical_ordering.to_vec(); + } + let (can_skip_sorting, should_reverse) = can_skip_sort( + window_expr[0].partition_by(), + required_ordering, + &sort_input.schema(), + physical_ordering, + )?; + // TODO: It seems like we exit early if even one path says we can + // not skip sorting. Therefore, I added an early return here + // instead of collecting these in a vector and calling "all". + // Please verify whether this is correct. + if !can_skip_sorting { + return Ok(None); + } + if let Some(first_should_reverse) = first_should_reverse { + if first_should_reverse != should_reverse { + return Ok(None); + } + } else { + first_should_reverse = Some(should_reverse); + } } else { - // If there is no physical ordering, there is no way to remove a sort -- immediately return: + // If there is no physical ordering, there is no way to remove a + // sort, so immediately return. return Ok(None); - }; - if physical_ordering.len() < physical_ordering_common.len() - || physical_ordering_common.is_empty() - { - physical_ordering_common = physical_ordering.to_vec(); } - let (can_skip_sorting, should_reverse) = can_skip_sort( - window_expr[0].partition_by(), - required_ordering, - &sort_input.schema(), - physical_ordering, - )?; - can_skip_sortings.push(can_skip_sorting); - should_reverses.push(should_reverse); } - let can_skip_sorting = can_skip_sortings.iter().all(|elem| *elem); - let can_skip_sorting = can_skip_sorting - && should_reverses + let new_window_expr = if first_should_reverse.unwrap() { + window_expr .iter() - .all(|elem| *elem == should_reverses[0]); - let should_reverse = should_reverses[0]; - if can_skip_sorting { - let new_window_expr = if should_reverse { - window_expr - .iter() - .map(|e| e.get_reverse_expr()) - .collect::>>() + .map(|e| e.get_reverse_expr()) + .collect::>>() + } else { + Some(window_expr.to_vec()) + }; + if let Some(window_expr) = new_window_expr { + let new_child = remove_corresponding_sort_from_sub_plan(sort_tree)?; + let new_schema = new_child.schema(); + + let uses_bounded_memory = window_expr.iter().all(|e| e.uses_bounded_memory()); + // If all window expressions can run with bounded memory, choose the + // bounded window variant: + let new_plan = if uses_bounded_memory { + Arc::new(BoundedWindowAggExec::try_new( + window_expr, + new_child, + new_schema, + partition_keys.to_vec(), + Some(physical_ordering_common), + )?) as _ } else { - Some(window_expr.to_vec()) + Arc::new(WindowAggExec::try_new( + window_expr, + new_child, + new_schema, + partition_keys.to_vec(), + Some(physical_ordering_common), + )?) as _ }; - if let Some(window_expr) = new_window_expr { - let new_child = remove_corresponding_sort_from_sub_plan(sort_onward)?; - let new_schema = new_child.schema(); - - let uses_bounded_memory = window_expr.iter().all(|e| e.uses_bounded_memory()); - // If all window exprs can run with bounded memory choose bounded window variant - let new_plan = if uses_bounded_memory { - Arc::new(BoundedWindowAggExec::try_new( - window_expr, - new_child, - new_schema, - partition_keys.to_vec(), - Some(physical_ordering_common), - )?) as _ - } else { - Arc::new(WindowAggExec::try_new( - window_expr, - new_child, - new_schema, - partition_keys.to_vec(), - Some(physical_ordering_common), - )?) as _ - }; - return Ok(Some(PlanWithCorrespondingSort::new(new_plan))); - } + return Ok(Some(PlanWithCorrespondingSort::new(new_plan))); } Ok(None) } -/// Updates child to remove the unnecessary coalesce partitions below it. +/// Updates child to remove the unnecessary `CoalescePartitions` below it. fn update_child_to_change_coalesce( child: &mut Arc, coalesce_onwards: &mut Option, @@ -620,40 +649,41 @@ fn update_child_to_change_coalesce( Ok(()) } -/// Removes the coalesce from the plan in `coalesce_onwards`. +/// Removes the `CoalescePartitions` from the plan in `coalesce_onwards`. fn change_corresponding_coalesce_in_sub_plan( coalesce_onwards: &mut ExecTree, sort_exec: Option<&SortExec>, ) -> Result> { - let res = if coalesce_onwards - .plan - .as_any() - .is::() - { - let mut coalesce_input = coalesce_onwards.plan.children()[0].clone(); - if let Some(sort_exec) = sort_exec { - let sort_expr = sort_exec.expr(); - if !ordering_satisfy( - coalesce_input.output_ordering(), - Some(sort_expr), - || sort_exec.equivalence_properties(), - ) { - coalesce_input = - add_sort_above_child(&coalesce_input, sort_expr.to_vec())?; + Ok( + if coalesce_onwards + .plan + .as_any() + .is::() + { + // We can safely use the 0th index since we have a `CoalescePartitionsExec`. + let coalesce_input = coalesce_onwards.plan.children()[0].clone(); + if let Some(sort_exec) = sort_exec { + let sort_expr = sort_exec.expr(); + // TODO: If coalesce is far below sort, can we still use its equivalence properties? + if !ordering_satisfy( + coalesce_input.output_ordering(), + Some(sort_expr), + || sort_exec.equivalence_properties(), + ) { + return add_sort_above_child(&coalesce_input, sort_expr.to_vec()); + } } - } - coalesce_input - } else { - let res = coalesce_onwards.plan.clone(); - let mut children = res.children(); - for elem in &mut coalesce_onwards.children { - children[elem.idx] = - change_corresponding_coalesce_in_sub_plan(elem, sort_exec)?; - } - res.with_new_children(children)? - }; - - Ok(res) + coalesce_input + } else { + let plan = coalesce_onwards.plan.clone(); + let mut children = plan.children(); + for item in &mut coalesce_onwards.children { + children[item.idx] = + change_corresponding_coalesce_in_sub_plan(item, sort_exec)?; + } + plan.with_new_children(children)? + }, + ) } /// Updates child to remove the unnecessary sorting below it. @@ -671,18 +701,16 @@ fn update_child_to_remove_unnecessary_sort( fn remove_corresponding_sort_from_sub_plan( sort_onwards: &mut ExecTree, ) -> Result> { - let res = if is_sort(&sort_onwards.plan) { - sort_onwards.plan.children()[0].clone() + if is_sort(&sort_onwards.plan) { + Ok(sort_onwards.plan.children()[0].clone()) } else { - let res = sort_onwards.plan.clone(); - let mut children = res.children(); - for elem in &mut sort_onwards.children { - children[elem.idx] = remove_corresponding_sort_from_sub_plan(elem)?; + let plan = sort_onwards.plan.clone(); + let mut children = plan.children(); + for item in &mut sort_onwards.children { + children[item.idx] = remove_corresponding_sort_from_sub_plan(item)?; } - res.with_new_children(children)? - }; - - Ok(res) + plan.with_new_children(children) + } } /// Updates child to modify the unnecessarily fine sorting below it. @@ -702,20 +730,18 @@ fn change_finer_sort_in_sub_plan( sort_onwards: &mut ExecTree, n_sort_expr: usize, ) -> Result> { - let res = if is_sort(&sort_onwards.plan) { - let prev_layer = sort_onwards.plan.children()[0].clone(); - let new_sort_expr = get_sort_exprs(&sort_onwards.plan)?[0..n_sort_expr].to_vec(); - add_sort_above_child(&prev_layer, new_sort_expr)? + let plan = &sort_onwards.plan; + if is_sort(plan) { + let prev_layer = plan.children()[0].clone(); + let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); + add_sort_above_child(&prev_layer, new_sort_expr) } else { - let res = sort_onwards.plan.clone(); - let mut children = res.children(); - for elem in &mut sort_onwards.children { - children[elem.idx] = change_finer_sort_in_sub_plan(elem, n_sort_expr)?; + let mut children = plan.children(); + for item in &mut sort_onwards.children { + children[item.idx] = change_finer_sort_in_sub_plan(item, n_sort_expr)?; } - res.with_new_children(children)? - }; - - Ok(res) + plan.clone().with_new_children(children) + } } /// Converts an [ExecutionPlan] trait object to a [PhysicalSortExpr] slice when possible. From 1fa2c43083dfe0d110929058c32ad89b0d2b25e0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 1 Feb 2023 10:11:35 +0300 Subject: [PATCH 11/31] Update todos --- .../src/physical_optimizer/sort_enforcement.rs | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 41c9a015c7d15..061b0c8efa658 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -152,8 +152,6 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { plan: item.plan, children: vec![], }); - // TODO: I moved this check here, I think you can exit early like this. - // Please double check. } else if is_limit(&item.plan) { // There is no sort linkage for this path, it starts at a limit. return None; @@ -461,12 +459,6 @@ fn ensure_sorting( return Ok(Some(result)); } } - // TODO: The following is an old note. Is it still true? Please double check - // and we can discuss at our next meeting. - // TODO: Once we can ensure that required ordering information propagates with - // necessary lineage information, compare `sort_input_ordering` and `required_ordering`. - // This will enable us to handle cases such as (a,b) -> Sort -> (a,b,c) -> Required(a,b). - // Currently, we can not remove such sorts. } } (Some(required), None) => { @@ -563,6 +555,11 @@ fn analyze_window_sort_removal( // Therefore, we can use the 0th index without loss of generality. let sort_input = sort_any.children()[0].clone(); let physical_ordering = sort_input.output_ordering(); + // TODO: Once we can ensure that required ordering information propagates with + // necessary lineage information, compare `physical_ordering` and required ordering by + // Window executor instead of `sort_output_ordering`. + // This will enable us to handle cases such as (a,b) -> Sort -> (a,b,c) -> Required(a,b). + // Currently, we can not remove such sorts. let required_ordering = sort_output_ordering.ok_or_else(|| { DataFusionError::Plan("A SortExec should have output ordering".to_string()) })?; @@ -580,10 +577,6 @@ fn analyze_window_sort_removal( &sort_input.schema(), physical_ordering, )?; - // TODO: It seems like we exit early if even one path says we can - // not skip sorting. Therefore, I added an early return here - // instead of collecting these in a vector and calling "all". - // Please verify whether this is correct. if !can_skip_sorting { return Ok(None); } From cb0debd547d6c64dd1105d453d011ffdcf80272c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 1 Feb 2023 11:10:48 +0300 Subject: [PATCH 12/31] minor changes --- datafusion/core/tests/sql/window.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 5f6737e45d6db..af24de4f3592d 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2465,6 +2465,7 @@ async fn test_window_agg_with_global_limit() -> Result<()> { Ok(()) } +#[tokio::test] async fn test_window_agg_low_cardinality() -> Result<()> { let config = SessionConfig::new().with_target_partitions(32); let ctx = SessionContext::with_config(config); From f3542a24794083cd0ecbb6fa177730936cb47888 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 1 Feb 2023 11:21:35 +0300 Subject: [PATCH 13/31] Add test for union doesn't maintain any of its child ordering --- .../physical_optimizer/sort_enforcement.rs | 58 ++++++++++++++++++- 1 file changed, 57 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 061b0c8efa658..0ee155b33f0d7 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -687,6 +687,7 @@ fn update_child_to_remove_unnecessary_sort( if let Some(sort_onwards) = sort_onwards { *child = remove_corresponding_sort_from_sub_plan(sort_onwards)?; } + *sort_onwards = None; Ok(()) } @@ -727,7 +728,13 @@ fn change_finer_sort_in_sub_plan( if is_sort(plan) { let prev_layer = plan.children()[0].clone(); let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); - add_sort_above_child(&prev_layer, new_sort_expr) + let updated_plan = add_sort_above_child(&prev_layer, new_sort_expr)?; + *sort_onwards = ExecTree { + idx: sort_onwards.idx, + children: vec![], + plan: updated_plan.clone(), + }; + Ok(updated_plan) } else { let mut children = plan.children(); for item in &mut sort_onwards.children { @@ -1339,6 +1346,55 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_union_inputs_different_sorted5() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![ + sort_expr("nullable_col", &schema), + sort_expr_options( + "non_nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]; + let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + let sort2 = sort_exec(sort_exprs2, source1); + + let union = union_exec(vec![sort1, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + + // Union doesn't preserve any of the inputs ordering. However, we should be able to change unnecessarily fine + // SortExecs under UnionExec with required SortExecs that are absolutely necessary. + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_multilayer_coalesce_partitions() -> Result<()> { let schema = create_test_schema()?; From 51ced7601e0d3103447f9ae4f0b7654794a4f668 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 1 Feb 2023 16:53:10 +0300 Subject: [PATCH 14/31] Add new test --- .../physical_optimizer/sort_enforcement.rs | 66 ++++++++++++++++--- 1 file changed, 56 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 0ee155b33f0d7..d381e38a2b55f 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -158,17 +158,17 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { } let plan = &item.plan; let union = plan.as_any().is::(); + // if executor is UnionExec and it has an output ordering its children + // maintains or partially maintains ordering. In either case propagate Sort linkage + // above + let partially_maintains = union && plan.output_ordering().is_some(); let flags = plan.maintains_input_order(); let required_orderings = plan.required_input_ordering(); let children = izip!(flags, item.sort_onwards, required_orderings) .filter_map(|(maintains, element, required_ordering)| { - // TODO: Why would we check if the plan is a Union every time in the closure? - // I moved it outside. Also, unless I am missing something, if the plan - // is a union, it maintains ordering (for any input) *if and only if* it - // has some output ordering. Assuming that this is indeed true, I removed - // the "plan.output_ordering.is_some()" predicate. Please double check if - // this is correct. - if required_ordering.is_none() && (maintains || union) { + if required_ordering.is_none() + && (maintains || partially_maintains) + { element } else { None @@ -564,8 +564,6 @@ fn analyze_window_sort_removal( DataFusionError::Plan("A SortExec should have output ordering".to_string()) })?; if let Some(physical_ordering) = physical_ordering { - // TODO: Do we need a longest common prefix approach here too? - // Please double check this logic is indeed correct. if physical_ordering_common.is_empty() || physical_ordering.len() < physical_ordering_common.len() { @@ -1373,7 +1371,8 @@ mod tests { let union = union_exec(vec![sort1, sort2]); let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); - // Union doesn't preserve any of the inputs ordering. However, we should be able to change unnecessarily fine + // Union doesn't preserve any of the inputs ordering in the example below. + // However, we should be able to change unnecessarily fine // SortExecs under UnionExec with required SortExecs that are absolutely necessary. let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", @@ -1395,6 +1394,53 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_window_multi_path_sort() -> Result<()> { + let schema = create_test_schema()?; + + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + // reverse sorting of sort_exprs2 + let sort_exprs3 = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source1 = parquet_exec_sorted(&schema, sort_exprs1); + let source2 = parquet_exec_sorted(&schema, sort_exprs2); + let sort1 = sort_exec(sort_exprs3.clone(), source1); + let sort2 = sort_exec(sort_exprs3.clone(), source2); + + let union = union_exec(vec![sort1, sort2]); + let physical_plan = window_exec("nullable_col", sort_exprs3, union); + + // WindowAggExec gets its Sorting from multiple children + // During removal of SortExecs it should be able to remove corresponding SortExecs + // Also inputs of the SortExecs are not necessarily same to be able to remove them. + let expected_input = vec![ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " UnionExec", + " SortExec: [nullable_col@0 DESC NULLS LAST]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 DESC NULLS LAST]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + ]; + let expected_optimized = vec![ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_multilayer_coalesce_partitions() -> Result<()> { let schema = create_test_schema()?; From d5d49432eee4cbb9f44357f8c9399a9e047838f7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 1 Feb 2023 17:20:14 +0300 Subject: [PATCH 15/31] use corresponding idx instead of 0th index --- .../physical_optimizer/sort_enforcement.rs | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index d381e38a2b55f..5443e781b0899 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -258,22 +258,22 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { plan, children: vec![], }) - } else if plan.children().is_empty() - || matches!( - // TODO: Please double check if using the first element here is a - // problem or not. Can `plan` not have multiple children? - plan.required_input_distribution()[0], - Distribution::SinglePartition - ) - { - // If the plan has no children, or it doesn't require a - // single partition, there is no linkage. + } else if plan.children().is_empty() { + // plan has no children, there is nothing + // to propagate None } else { let children = item .coalesce_onwards .into_iter() .flatten() + .filter(|elem| { + // doesn't require single partition + !matches!( + plan.required_input_distribution()[elem.idx], + Distribution::SinglePartition + ) + }) .collect::>(); if children.is_empty() { None From 1359a73fc6306fa38f0538bba7b3bbf7a8c3f37f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 1 Feb 2023 17:41:03 +0300 Subject: [PATCH 16/31] Add global limit test --- .../physical_optimizer/sort_enforcement.rs | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 5443e781b0899..faa235104e435 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -1134,6 +1134,52 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_do_not_remove_sort_with_limit() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort = sort_exec(sort_exprs.clone(), source1); + let limit = limit_exec(sort); + + let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); + + let union = union_exec(vec![source2, limit]); + let repartition = repartition_exec(union); + let physical_plan = sort_preserving_merge_exec(sort_exprs, repartition); + + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + + // expect to keep the bottom SortExec + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_change_wrong_sorting() -> Result<()> { let schema = create_test_schema()?; @@ -1653,4 +1699,15 @@ mod tests { fn union_exec(input: Vec>) -> Arc { Arc::new(UnionExec::new(input)) } + + fn limit_exec(input: Arc) -> Arc { + let new_input = Arc::new(LocalLimitExec::new(input, 100)); + Arc::new(GlobalLimitExec::new(new_input, 0, Some(100))) + } + + fn repartition_exec(input: Arc) -> Arc { + Arc::new( + RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap(), + ) + } } From a470f9827f97b6ae541df2dc4a65adc298a59944 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 2 Feb 2023 12:44:49 +0300 Subject: [PATCH 17/31] Add SortPreservingMerge handling --- .../physical_optimizer/sort_enforcement.rs | 190 +++++++++++++++--- 1 file changed, 167 insertions(+), 23 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index faa235104e435..1c5bd7e1815ec 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -141,23 +141,24 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { .into_iter() .enumerate() .map(|(idx, item)| { + let plan = &item.plan; // Leaves of the `sort_onwards` tree are sort-introducing operators // (e.g `SortExec`, `SortPreservingMergeExec`). This tree collects // all the intermediate executors that maintain this ordering. If // we just saw a sort-introducing operator, we reset the tree and // start accumulating. - if is_sort(&item.plan) { + if is_sort_exec(plan) { return Some(ExecTree { idx, plan: item.plan, children: vec![], }); - } else if is_limit(&item.plan) { + } else if is_limit(plan) { // There is no sort linkage for this path, it starts at a limit. return None; } - let plan = &item.plan; let union = plan.as_any().is::(); + let is_sort_preserving_merge = is_sort_preserving_merge_exec(plan); // if executor is UnionExec and it has an output ordering its children // maintains or partially maintains ordering. In either case propagate Sort linkage // above @@ -166,8 +167,9 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { let required_orderings = plan.required_input_ordering(); let children = izip!(flags, item.sort_onwards, required_orderings) .filter_map(|(maintains, element, required_ordering)| { - if required_ordering.is_none() - && (maintains || partially_maintains) + if (required_ordering.is_none() + && (maintains || partially_maintains)) + || is_sort_preserving_merge { element } else { @@ -389,13 +391,16 @@ fn is_limit(plan: &Arc) -> bool { plan.as_any().is::() || plan.as_any().is::() } -/// Checks whether the given executor is sort-introducing. -fn is_sort(plan: &Arc) -> bool { - // TODO: Add support for SortPreservingMergeExec also. - // This was your comment from before, let's fix it. +/// Checks whether the given executor is SortExec. +fn is_sort_exec(plan: &Arc) -> bool { plan.as_any().is::() } +/// Checks whether the given executor is SortPreservingMergeExec. +fn is_sort_preserving_merge_exec(plan: &Arc) -> bool { + plan.as_any().is::() +} + /// This function enforces sorting requirements and makes optimizations without /// violating these requirements whenever possible. fn ensure_sorting( @@ -428,7 +433,7 @@ fn ensure_sorting( ); if !is_ordering_satisfied { // Make sure we preserve the ordering requirements: - update_child_to_remove_unnecessary_sort(child, sort_onwards)?; + update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; let sort_expr = required_ordering.to_vec(); *child = add_sort_above_child(child, sort_expr)?; *sort_onwards = Some(ExecTree { @@ -445,6 +450,7 @@ fn ensure_sorting( exec.window_expr(), &exec.partition_keys, tree, + &plan, )? { return Ok(Some(result)); } @@ -455,6 +461,7 @@ fn ensure_sorting( exec.window_expr(), &exec.partition_keys, tree, + &plan, )? { return Ok(Some(result)); } @@ -475,10 +482,14 @@ fn ensure_sorting( // another order-imposing operator. Remove or update this sort: if !plan.maintains_input_order()[idx] { let count = plan.output_ordering().map_or(0, |e| e.len()); - if (count > 0) && !is_sort(&plan) { + if (count > 0) && !is_sort_exec(&plan) { update_child_to_change_finer_sort(child, sort_onwards, count)?; } else { - update_child_to_remove_unnecessary_sort(child, sort_onwards)?; + update_child_to_remove_unnecessary_sort( + child, + sort_onwards, + &plan, + )?; } } } @@ -545,6 +556,8 @@ fn analyze_window_sort_removal( window_expr: &[Arc], partition_keys: &[Arc], sort_tree: &mut ExecTree, + // Either BoundedWindowAggExec or WindowAggExec + window_exec: &Arc, ) -> Result> { let mut first_should_reverse = None; let mut physical_ordering_common = vec![]; @@ -600,7 +613,14 @@ fn analyze_window_sort_removal( Some(window_expr.to_vec()) }; if let Some(window_expr) = new_window_expr { - let new_child = remove_corresponding_sort_from_sub_plan(sort_tree)?; + let requires_single_partition = matches!( + window_exec.required_input_distribution()[sort_tree.idx], + Distribution::SinglePartition + ); + let new_child = remove_corresponding_sort_from_sub_plan( + sort_tree, + requires_single_partition, + )?; let new_schema = new_child.schema(); let uses_bounded_memory = window_expr.iter().all(|e| e.uses_bounded_memory()); @@ -681,9 +701,17 @@ fn change_corresponding_coalesce_in_sub_plan( fn update_child_to_remove_unnecessary_sort( child: &mut Arc, sort_onwards: &mut Option, + parent: &Arc, ) -> Result<()> { if let Some(sort_onwards) = sort_onwards { - *child = remove_corresponding_sort_from_sub_plan(sort_onwards)?; + let requires_single_partition = matches!( + parent.required_input_distribution()[sort_onwards.idx], + Distribution::SinglePartition + ); + *child = remove_corresponding_sort_from_sub_plan( + sort_onwards, + requires_single_partition, + )?; } *sort_onwards = None; Ok(()) @@ -692,16 +720,35 @@ fn update_child_to_remove_unnecessary_sort( /// Removes the sort from the plan in `sort_onwards`. fn remove_corresponding_sort_from_sub_plan( sort_onwards: &mut ExecTree, + requires_single_partition: bool, ) -> Result> { - if is_sort(&sort_onwards.plan) { + // SortExec is always at the bottom of the tree + if is_sort_exec(&sort_onwards.plan) { Ok(sort_onwards.plan.children()[0].clone()) + } else if is_sort_preserving_merge_exec(&sort_onwards.plan) { + let new_plan = remove_corresponding_sort_from_sub_plan( + &mut sort_onwards.children[0], + false, + )?; + if requires_single_partition + && new_plan.output_partitioning().partition_count() > 1 + { + Ok(Arc::new(CoalescePartitionsExec::new(new_plan))) + } else { + Ok(new_plan) + } } else { - let plan = sort_onwards.plan.clone(); + let plan = &sort_onwards.plan; let mut children = plan.children(); for item in &mut sort_onwards.children { - children[item.idx] = remove_corresponding_sort_from_sub_plan(item)?; + let requires_single_partition = matches!( + plan.required_input_distribution()[item.idx], + Distribution::SinglePartition + ); + children[item.idx] = + remove_corresponding_sort_from_sub_plan(item, requires_single_partition)?; } - plan.with_new_children(children) + plan.clone().with_new_children(children) } } @@ -723,7 +770,7 @@ fn change_finer_sort_in_sub_plan( n_sort_expr: usize, ) -> Result> { let plan = &sort_onwards.plan; - if is_sort(plan) { + if is_sort_exec(plan) { let prev_layer = plan.children()[0].clone(); let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); let updated_plan = add_sort_above_child(&prev_layer, new_sort_expr)?; @@ -852,6 +899,8 @@ mod tests { use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; use crate::physical_optimizer::dist_enforcement::EnforceDistribution; + use crate::physical_plan::aggregates::PhysicalGroupBy; + use crate::physical_plan::aggregates::{AggregateExec, AggregateMode}; use crate::physical_plan::file_format::{FileScanConfig, ParquetExec}; use crate::physical_plan::filter::FilterExec; use crate::physical_plan::memory::MemoryExec; @@ -1134,6 +1183,83 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_remove_unnecessary_sort2() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort2 = sort_exec(sort_exprs.clone(), spm); + let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); + + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort3 = sort_exec(sort_exprs, spm2); + let physical_plan = repartition_exec(repartition_exec(sort3)); + + let expected_input = vec![ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: [nullable_col@0 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: [non_nullable_col@1 ASC]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + + let expected_optimized = vec![ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_sort3() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let repartition_exec = repartition_exec(spm); + let sort2 = sort_exec(sort_exprs.clone(), repartition_exec); + let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); + + let physical_plan = aggregate_exec(spm2); + + let expected_input = vec![ + "AggregateExec: mode=Final, gby=[], aggr=[]", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: [non_nullable_col@1 ASC]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + + let expected_optimized = vec![ + "AggregateExec: mode=Final, gby=[], aggr=[]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_do_not_remove_sort_with_limit() -> Result<()> { let schema = create_test_schema()?; @@ -1144,7 +1270,8 @@ mod tests { sort_expr("non_nullable_col", &schema), ]; let sort = sort_exec(sort_exprs.clone(), source1); - let limit = limit_exec(sort); + let limit = local_limit_exec(sort); + let limit = global_limit_exec(limit); let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); @@ -1700,9 +1827,12 @@ mod tests { Arc::new(UnionExec::new(input)) } - fn limit_exec(input: Arc) -> Arc { - let new_input = Arc::new(LocalLimitExec::new(input, 100)); - Arc::new(GlobalLimitExec::new(new_input, 0, Some(100))) + fn local_limit_exec(input: Arc) -> Arc { + Arc::new(LocalLimitExec::new(input, 100)) + } + + fn global_limit_exec(input: Arc) -> Arc { + Arc::new(GlobalLimitExec::new(input, 0, Some(100))) } fn repartition_exec(input: Arc) -> Arc { @@ -1710,4 +1840,18 @@ mod tests { RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap(), ) } + + fn aggregate_exec(input: Arc) -> Arc { + let schema = input.schema(); + Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + vec![], + input, + schema, + ) + .unwrap(), + ) + } } From 027eacefcf21cbe489aa33f596067b8dd4b198ac Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 2 Feb 2023 13:46:18 +0300 Subject: [PATCH 18/31] add finer sorting change Sort and SortPreserve test --- .../physical_optimizer/sort_enforcement.rs | 101 +++++++++++++----- 1 file changed, 77 insertions(+), 24 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 1c5bd7e1815ec..f7883bede0494 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -44,7 +44,6 @@ use crate::physical_plan::{with_new_children_if_necessary, Distribution, Executi use arrow::datatypes::SchemaRef; use datafusion_common::{reverse_sort_options, DataFusionError}; use datafusion_physical_expr::utils::{ordering_satisfy, ordering_satisfy_concrete}; -use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use itertools::{concat, izip}; use std::iter::zip; @@ -445,24 +444,10 @@ fn ensure_sorting( if let Some(tree) = sort_onwards { // For window expressions, we can remove some sorts when we can // calculate the result in reverse: - if let Some(exec) = plan.as_any().downcast_ref::() { - if let Some(result) = analyze_window_sort_removal( - exec.window_expr(), - &exec.partition_keys, - tree, - &plan, - )? { - return Ok(Some(result)); - } - } else if let Some(exec) = - plan.as_any().downcast_ref::() + if plan.as_any().is::() + || plan.as_any().is::() { - if let Some(result) = analyze_window_sort_removal( - exec.window_expr(), - &exec.partition_keys, - tree, - &plan, - )? { + if let Some(result) = analyze_window_sort_removal(tree, &plan)? { return Ok(Some(result)); } } @@ -553,12 +538,22 @@ fn analyze_immediate_sort_removal( /// Analyzes a [WindowAggExec] or a [BoundedWindowAggExec] to determine whether /// it may allow removing a sort. fn analyze_window_sort_removal( - window_expr: &[Arc], - partition_keys: &[Arc], sort_tree: &mut ExecTree, // Either BoundedWindowAggExec or WindowAggExec window_exec: &Arc, ) -> Result> { + let (window_expr, partition_keys) = if let Some(exec) = + window_exec.as_any().downcast_ref::() + { + (exec.window_expr(), &exec.partition_keys) + } else if let Some(exec) = window_exec.as_any().downcast_ref::() { + (exec.window_expr(), &exec.partition_keys) + } else { + return Err(DataFusionError::Execution( + "Expects to receive either WindowAggExec of BoundedWindowAggExec".to_string(), + )); + }; + let mut first_should_reverse = None; let mut physical_ordering_common = vec![]; for sort_any in sort_tree.get_leaves() { @@ -780,6 +775,14 @@ fn change_finer_sort_in_sub_plan( plan: updated_plan.clone(), }; Ok(updated_plan) + } else if is_sort_preserving_merge_exec(plan) { + let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); + let new_plan = + change_finer_sort_in_sub_plan(&mut sort_onwards.children[0], n_sort_expr)?; + let updated_plan = Arc::new(SortPreservingMergeExec::new(new_sort_expr, new_plan)) + as Arc; + sort_onwards.plan = updated_plan.clone(); + Ok(updated_plan) } else { let mut children = plan.children(); for item in &mut sort_onwards.children { @@ -1492,9 +1495,9 @@ mod tests { let union = union_exec(vec![sort1, source2, sort2]); let physical_plan = sort_preserving_merge_exec(sort_exprs1, union); - // First input to the union is not Sorted (SortExec is finer than required ordering by the SortPreservingMergeExec above). - // Second input to the union is already Sorted (matches with the required ordering by the SortPreservingMergeExec above). - // Third input to the union is not Sorted (SortExec is matches required ordering by the SortPreservingMergeExec above). + // Requirement of SortPreservingMergeExec is not satisfied. + // Should add required SortExec to satisfy it. Then should remove unnecessary ordering below + // UnionExec where SortExec comes from multiple children. let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " UnionExec", @@ -1504,7 +1507,6 @@ mod tests { " SortExec: [nullable_col@0 ASC]", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; - // should adjust sorting in the first input of the union such that it is not unnecessarily fine let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", @@ -1567,6 +1569,57 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_union_inputs_different_sorted6() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + let sort_exprs2 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let repartition = repartition_exec(source1); + let sort2 = sort_exec(sort_exprs2.clone(), repartition); + let spm = sort_preserving_merge_exec(sort_exprs2, sort2); + + let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); + + let union = union_exec(vec![sort1, source2, spm]); + let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); + + // This input of UnionExec is unnecessarily fine than the required ordering by SortPreservingMergeExec + // Ordering of that child should be changed with requirement ordering(SortEXec and SortPreservingMergeExec) + // at that path should be updated. + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + // should adjust sorting in the third input of the union such that it is not unnecessarily fine + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_window_multi_path_sort() -> Result<()> { let schema = create_test_schema()?; From ec272544db3b79f00ae20d5c1f383ebb3b888b31 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 2 Feb 2023 14:11:34 +0300 Subject: [PATCH 19/31] Update test --- .../src/physical_optimizer/sort_enforcement.rs | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index f7883bede0494..0a096569adb19 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -1581,8 +1581,7 @@ mod tests { sort_expr("non_nullable_col", &schema), ]; let repartition = repartition_exec(source1); - let sort2 = sort_exec(sort_exprs2.clone(), repartition); - let spm = sort_preserving_merge_exec(sort_exprs2, sort2); + let spm = sort_preserving_merge_exec(sort_exprs2, repartition); let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); @@ -1590,9 +1589,10 @@ mod tests { let union = union_exec(vec![sort1, source2, spm]); let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); - // This input of UnionExec is unnecessarily fine than the required ordering by SortPreservingMergeExec - // Ordering of that child should be changed with requirement ordering(SortEXec and SortPreservingMergeExec) - // at that path should be updated. + // Third child of UnionExec is unnecessarily fine than the required ordering by SortPreservingMergeExec at the top, + // Also plan is not valid as it is. SortPreservingMergeExec under the third child of the + // UnionExec should have its input ordered. We should add SortExec below it. In the final plan + // Ordering of the third child shouldn't be finer than necessary let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", @@ -1600,9 +1600,8 @@ mod tests { " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; // should adjust sorting in the third input of the union such that it is not unnecessarily fine let expected_optimized = vec![ From 4c32040bf7f566c5ead0798e3eb19f98639c71bc Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 2 Feb 2023 15:47:07 +0300 Subject: [PATCH 20/31] simplifications --- .../physical_optimizer/sort_enforcement.rs | 46 ++++++++++--------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 0a096569adb19..d46582c106511 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -720,18 +720,6 @@ fn remove_corresponding_sort_from_sub_plan( // SortExec is always at the bottom of the tree if is_sort_exec(&sort_onwards.plan) { Ok(sort_onwards.plan.children()[0].clone()) - } else if is_sort_preserving_merge_exec(&sort_onwards.plan) { - let new_plan = remove_corresponding_sort_from_sub_plan( - &mut sort_onwards.children[0], - false, - )?; - if requires_single_partition - && new_plan.output_partitioning().partition_count() > 1 - { - Ok(Arc::new(CoalescePartitionsExec::new(new_plan))) - } else { - Ok(new_plan) - } } else { let plan = &sort_onwards.plan; let mut children = plan.children(); @@ -743,7 +731,18 @@ fn remove_corresponding_sort_from_sub_plan( children[item.idx] = remove_corresponding_sort_from_sub_plan(item, requires_single_partition)?; } - plan.clone().with_new_children(children) + if is_sort_preserving_merge_exec(plan) { + let child = &children[0]; + if requires_single_partition + && child.output_partitioning().partition_count() > 1 + { + Ok(Arc::new(CoalescePartitionsExec::new(child.clone()))) + } else { + Ok(child.clone()) + } + } else { + plan.clone().with_new_children(children) + } } } @@ -765,6 +764,7 @@ fn change_finer_sort_in_sub_plan( n_sort_expr: usize, ) -> Result> { let plan = &sort_onwards.plan; + // SortExec is always at the bottom of the tree if is_sort_exec(plan) { let prev_layer = plan.children()[0].clone(); let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); @@ -775,20 +775,22 @@ fn change_finer_sort_in_sub_plan( plan: updated_plan.clone(), }; Ok(updated_plan) - } else if is_sort_preserving_merge_exec(plan) { - let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); - let new_plan = - change_finer_sort_in_sub_plan(&mut sort_onwards.children[0], n_sort_expr)?; - let updated_plan = Arc::new(SortPreservingMergeExec::new(new_sort_expr, new_plan)) - as Arc; - sort_onwards.plan = updated_plan.clone(); - Ok(updated_plan) } else { let mut children = plan.children(); for item in &mut sort_onwards.children { children[item.idx] = change_finer_sort_in_sub_plan(item, n_sort_expr)?; } - plan.clone().with_new_children(children) + if is_sort_preserving_merge_exec(plan) { + let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); + let updated_plan = Arc::new(SortPreservingMergeExec::new( + new_sort_expr, + children[0].clone(), + )) as Arc; + sort_onwards.plan = updated_plan.clone(); + Ok(updated_plan) + } else { + plan.clone().with_new_children(children) + } } } From d8e9515c70895e15f63799cd47a0d41f39341c8c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 2 Feb 2023 18:15:54 +0300 Subject: [PATCH 21/31] Simplifications --- .../physical_optimizer/sort_enforcement.rs | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index d46582c106511..0c9701e31ee90 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -670,11 +670,10 @@ fn change_corresponding_coalesce_in_sub_plan( let coalesce_input = coalesce_onwards.plan.children()[0].clone(); if let Some(sort_exec) = sort_exec { let sort_expr = sort_exec.expr(); - // TODO: If coalesce is far below sort, can we still use its equivalence properties? if !ordering_satisfy( coalesce_input.output_ordering(), Some(sort_expr), - || sort_exec.equivalence_properties(), + || coalesce_input.equivalence_properties(), ) { return add_sort_above_child(&coalesce_input, sort_expr.to_vec()); } @@ -1245,6 +1244,9 @@ mod tests { let physical_plan = aggregate_exec(spm2); + // When removing SortPreservingMergeExec make sure that partitioning requirements are + // not violated. In some cases we may need to replace it with CoalescePartitionsExec instead + // directly removing it. let expected_input = vec![ "AggregateExec: mode=Final, gby=[], aggr=[]", " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", @@ -1673,10 +1675,7 @@ mod tests { let schema = create_test_schema()?; let source1 = parquet_exec(&schema); - let repartition = Arc::new(RepartitionExec::try_new( - source1, - Partitioning::RoundRobinBatch(2), - )?) as Arc; + let repartition = repartition_exec(source1); let coalesce = Arc::new(CoalescePartitionsExec::new(repartition)) as _; // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before let filter = filter_exec( @@ -1695,14 +1694,14 @@ mod tests { "SortExec: [nullable_col@0 ASC]", " FilterExec: NOT non_nullable_col@1", " CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " FilterExec: NOT non_nullable_col@1", " SortExec: [nullable_col@0 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1724,10 +1723,7 @@ mod tests { let memory_exec = memory_exec(&schema); let sort_exprs = vec![sort_expr("nullable_col", &schema)]; let window = window_exec("nullable_col", sort_exprs.clone(), memory_exec); - let repartition = Arc::new(RepartitionExec::try_new( - window, - Partitioning::RoundRobinBatch(2), - )?) as Arc; + let repartition = repartition_exec(window); let orig_plan = Arc::new(SortExec::new_with_partitioning( sort_exprs, From 3f6870346a623ce08f57ea26f84268b24f76da20 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 2 Feb 2023 17:47:43 -0600 Subject: [PATCH 22/31] Improved comments and naming --- .../physical_optimizer/sort_enforcement.rs | 97 ++++++++++--------- 1 file changed, 51 insertions(+), 46 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 0c9701e31ee90..0e95b46c75eba 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -146,7 +146,7 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { // all the intermediate executors that maintain this ordering. If // we just saw a sort-introducing operator, we reset the tree and // start accumulating. - if is_sort_exec(plan) { + if is_sort(plan) { return Some(ExecTree { idx, plan: item.plan, @@ -156,19 +156,20 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { // There is no sort linkage for this path, it starts at a limit. return None; } - let union = plan.as_any().is::(); - let is_sort_preserving_merge = is_sort_preserving_merge_exec(plan); - // if executor is UnionExec and it has an output ordering its children - // maintains or partially maintains ordering. In either case propagate Sort linkage - // above - let partially_maintains = union && plan.output_ordering().is_some(); + let is_spm = is_sort_preserving_merge(plan); + let is_union = plan.as_any().is::(); + // If the executor is a `UnionExec`, and it has an output ordering; + // then it at least partially maintains some child's output ordering. + // Therefore, we propagate this information upwards. + let partially_maintains = + is_union && plan.output_ordering().is_some(); let flags = plan.maintains_input_order(); let required_orderings = plan.required_input_ordering(); let children = izip!(flags, item.sort_onwards, required_orderings) .filter_map(|(maintains, element, required_ordering)| { if (required_ordering.is_none() && (maintains || partially_maintains)) - || is_sort_preserving_merge + || is_spm { element } else { @@ -260,18 +261,18 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { children: vec![], }) } else if plan.children().is_empty() { - // plan has no children, there is nothing - // to propagate + // Plan has no children, there is nothing to propagate. None } else { let children = item .coalesce_onwards .into_iter() .flatten() - .filter(|elem| { - // doesn't require single partition + .filter(|item| { + // Only consider operators that don't require a + // single partition. !matches!( - plan.required_input_distribution()[elem.idx], + plan.required_input_distribution()[item.idx], Distribution::SinglePartition ) }) @@ -390,13 +391,13 @@ fn is_limit(plan: &Arc) -> bool { plan.as_any().is::() || plan.as_any().is::() } -/// Checks whether the given executor is SortExec. -fn is_sort_exec(plan: &Arc) -> bool { +/// Checks whether the given executor is a `SortExec`. +fn is_sort(plan: &Arc) -> bool { plan.as_any().is::() } -/// Checks whether the given executor is SortPreservingMergeExec. -fn is_sort_preserving_merge_exec(plan: &Arc) -> bool { +/// Checks whether the given executor is a `SortPreservingMergeExec`. +fn is_sort_preserving_merge(plan: &Arc) -> bool { plan.as_any().is::() } @@ -467,7 +468,7 @@ fn ensure_sorting( // another order-imposing operator. Remove or update this sort: if !plan.maintains_input_order()[idx] { let count = plan.output_ordering().map_or(0, |e| e.len()); - if (count > 0) && !is_sort_exec(&plan) { + if (count > 0) && !is_sort(&plan) { update_child_to_change_finer_sort(child, sort_onwards, count)?; } else { update_child_to_remove_unnecessary_sort( @@ -539,7 +540,6 @@ fn analyze_immediate_sort_removal( /// it may allow removing a sort. fn analyze_window_sort_removal( sort_tree: &mut ExecTree, - // Either BoundedWindowAggExec or WindowAggExec window_exec: &Arc, ) -> Result> { let (window_expr, partition_keys) = if let Some(exec) = @@ -549,7 +549,7 @@ fn analyze_window_sort_removal( } else if let Some(exec) = window_exec.as_any().downcast_ref::() { (exec.window_expr(), &exec.partition_keys) } else { - return Err(DataFusionError::Execution( + return Err(DataFusionError::Plan( "Expects to receive either WindowAggExec of BoundedWindowAggExec".to_string(), )); }; @@ -564,8 +564,8 @@ fn analyze_window_sort_removal( let sort_input = sort_any.children()[0].clone(); let physical_ordering = sort_input.output_ordering(); // TODO: Once we can ensure that required ordering information propagates with - // necessary lineage information, compare `physical_ordering` and required ordering by - // Window executor instead of `sort_output_ordering`. + // the necessary lineage information, compare `physical_ordering` and the + // ordering required by the window executor instead of `sort_output_ordering`. // This will enable us to handle cases such as (a,b) -> Sort -> (a,b,c) -> Required(a,b). // Currently, we can not remove such sorts. let required_ordering = sort_output_ordering.ok_or_else(|| { @@ -716,8 +716,8 @@ fn remove_corresponding_sort_from_sub_plan( sort_onwards: &mut ExecTree, requires_single_partition: bool, ) -> Result> { - // SortExec is always at the bottom of the tree - if is_sort_exec(&sort_onwards.plan) { + // A `SortExec` is always at the bottom of the tree. + if is_sort(&sort_onwards.plan) { Ok(sort_onwards.plan.children()[0].clone()) } else { let plan = &sort_onwards.plan; @@ -730,7 +730,7 @@ fn remove_corresponding_sort_from_sub_plan( children[item.idx] = remove_corresponding_sort_from_sub_plan(item, requires_single_partition)?; } - if is_sort_preserving_merge_exec(plan) { + if is_sort_preserving_merge(plan) { let child = &children[0]; if requires_single_partition && child.output_partitioning().partition_count() > 1 @@ -763,8 +763,8 @@ fn change_finer_sort_in_sub_plan( n_sort_expr: usize, ) -> Result> { let plan = &sort_onwards.plan; - // SortExec is always at the bottom of the tree - if is_sort_exec(plan) { + // A `SortExec` is always at the bottom of the tree. + if is_sort(plan) { let prev_layer = plan.children()[0].clone(); let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); let updated_plan = add_sort_above_child(&prev_layer, new_sort_expr)?; @@ -779,7 +779,7 @@ fn change_finer_sort_in_sub_plan( for item in &mut sort_onwards.children { children[item.idx] = change_finer_sort_in_sub_plan(item, n_sort_expr)?; } - if is_sort_preserving_merge_exec(plan) { + if is_sort_preserving_merge(plan) { let new_sort_expr = get_sort_exprs(plan)?[0..n_sort_expr].to_vec(); let updated_plan = Arc::new(SortPreservingMergeExec::new( new_sort_expr, @@ -1244,9 +1244,9 @@ mod tests { let physical_plan = aggregate_exec(spm2); - // When removing SortPreservingMergeExec make sure that partitioning requirements are - // not violated. In some cases we may need to replace it with CoalescePartitionsExec instead - // directly removing it. + // When removing a `SortPreservingMergeExec`, make sure that partitioning + // requirements are not violated. In some cases, we may need to replace + // it with a `CoalescePartitionsExec` instead of directly removing it. let expected_input = vec![ "AggregateExec: mode=Final, gby=[], aggr=[]", " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", @@ -1298,7 +1298,7 @@ mod tests { " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; - // expect to keep the bottom SortExec + // We should keep the bottom `SortExec`. let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", @@ -1499,9 +1499,10 @@ mod tests { let union = union_exec(vec![sort1, source2, sort2]); let physical_plan = sort_preserving_merge_exec(sort_exprs1, union); - // Requirement of SortPreservingMergeExec is not satisfied. - // Should add required SortExec to satisfy it. Then should remove unnecessary ordering below - // UnionExec where SortExec comes from multiple children. + // Ordering requirement of the `SortPreservingMergeExec` is not met. + // Should modify the plan to ensure that all three inputs to the + // `UnionExec` satisfy the ordering, OR add a single sort after + // the `UnionExec` (both of which are equally good for this example). let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " UnionExec", @@ -1550,9 +1551,9 @@ mod tests { let union = union_exec(vec![sort1, sort2]); let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); - // Union doesn't preserve any of the inputs ordering in the example below. - // However, we should be able to change unnecessarily fine - // SortExecs under UnionExec with required SortExecs that are absolutely necessary. + // The `UnionExec` doesn't preserve any of the inputs ordering in the + // example below. However, we should be able to change the unnecessarily + // fine `SortExec`s below with required `SortExec`s that are absolutely necessary. let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", @@ -1593,10 +1594,12 @@ mod tests { let union = union_exec(vec![sort1, source2, spm]); let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); - // Third child of UnionExec is unnecessarily fine than the required ordering by SortPreservingMergeExec at the top, - // Also plan is not valid as it is. SortPreservingMergeExec under the third child of the - // UnionExec should have its input ordered. We should add SortExec below it. In the final plan - // Ordering of the third child shouldn't be finer than necessary + // The plan is not valid as it is -- the input ordering requirement + // of the `SortPreservingMergeExec` under the third child of the + // `UnionExec` is not met. We should add a `SortExec` below it. + // At the same time, this ordering requirement is unnecessarily fine. + // The final plan should be valid AND the ordering of the third child + // shouldn't be finer than necessary. let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", @@ -1607,7 +1610,8 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; - // should adjust sorting in the third input of the union such that it is not unnecessarily fine + // Should adjust the requirement in the third input of the union so + // that it is not unnecessarily fine. let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", @@ -1649,9 +1653,10 @@ mod tests { let union = union_exec(vec![sort1, sort2]); let physical_plan = window_exec("nullable_col", sort_exprs3, union); - // WindowAggExec gets its Sorting from multiple children - // During removal of SortExecs it should be able to remove corresponding SortExecs - // Also inputs of the SortExecs are not necessarily same to be able to remove them. + // The `WindowAggExec` gets its sorting from multiple children jointly. + // During the removal of `SortExec`s, it should be able to remove the + // corresponding SortExecs together. Also, the inputs of these `SortExec`s + // are not necessarily the same to be able to remove them. let expected_input = vec![ "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", " UnionExec", From 13d90747061bb520395b9472c898ddb358a47b12 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 7 Feb 2023 10:26:29 +0300 Subject: [PATCH 23/31] Remove explicit union check --- .../physical_optimizer/sort_enforcement.rs | 42 +++++++++---------- 1 file changed, 19 insertions(+), 23 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 0e95b46c75eba..a4caaf09f07ae 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -38,7 +38,6 @@ use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::rewrite::TreeNodeRewritable; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; use arrow::datatypes::SchemaRef; @@ -135,14 +134,13 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { let children_plans = children_requirements .iter() .map(|item| item.plan.clone()) - .collect(); + .collect::>(); let sort_onwards = children_requirements .into_iter() .enumerate() .map(|(idx, item)| { let plan = &item.plan; - // Leaves of the `sort_onwards` tree are sort-introducing operators - // (e.g `SortExec`, `SortPreservingMergeExec`). This tree collects + // Leaves of the `sort_onwards` are `SortExec`(Introduces ordering). This tree collects // all the intermediate executors that maintain this ordering. If // we just saw a sort-introducing operator, we reset the tree and // start accumulating. @@ -157,26 +155,24 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { return None; } let is_spm = is_sort_preserving_merge(plan); - let is_union = plan.as_any().is::(); - // If the executor is a `UnionExec`, and it has an output ordering; - // then it at least partially maintains some child's output ordering. - // Therefore, we propagate this information upwards. - let partially_maintains = - is_union && plan.output_ordering().is_some(); - let flags = plan.maintains_input_order(); + let output_ordering = plan.output_ordering(); let required_orderings = plan.required_input_ordering(); - let children = izip!(flags, item.sort_onwards, required_orderings) - .filter_map(|(maintains, element, required_ordering)| { - if (required_ordering.is_none() - && (maintains || partially_maintains)) - || is_spm - { - element - } else { - None - } - }) - .collect::>(); + let children = + izip!(&plan.children(), item.sort_onwards, required_orderings) + .filter_map(|(child, element, required_ordering)| { + // Executor maintains or partially maintains its child's output ordering + let maintains = ordering_satisfy( + child.output_ordering(), + output_ordering, + || child.equivalence_properties(), + ); + if (required_ordering.is_none() && maintains) || is_spm { + element + } else { + None + } + }) + .collect::>(); if !children.is_empty() { // Add parent node to the tree if there is at least one // child with a subtree: From fdffa3f88e0e0bdb2048ee1e3d0104aa0b6a8454 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 7 Feb 2023 11:25:48 +0300 Subject: [PATCH 24/31] make parallelize sort flag config parameter --- datafusion/common/src/config.rs | 11 +++++ datafusion/core/src/execution/context.rs | 18 +++++-- .../physical_optimizer/dist_enforcement.rs | 2 +- .../src/physical_optimizer/repartition.rs | 2 +- .../physical_optimizer/sort_enforcement.rs | 29 ++++++----- datafusion/core/tests/sql/window.rs | 48 +++++++++++++++++-- .../test_files/information_schema.slt | 3 +- docs/source/user-guide/configs.md | 1 + 8 files changed, 89 insertions(+), 25 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 661109416092a..1037502c6e34a 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -290,6 +290,17 @@ config_namespace! { /// functions in parallel using the provided `target_partitions` level" pub repartition_windows: bool, default = true + /// Should DataFusion parallelize Sort during physical plan creation. + /// with this flag is enabled, plans in the form below + /// "SortExec: [a@0 ASC]", + /// " CoalescePartitionsExec", + /// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + /// would turn into the plan below which performs better in multithreaded environments + /// "SortPreservingMergeExec: [a@0 ASC]", + /// " SortExec: [a@0 ASC]", + /// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + pub parallelize_sorts: bool, default = true + /// When set to true, the logical plan optimizer will produce warning /// messages if any optimization rules produce errors and then proceed to the next /// rule. When set to false, any rules that produce errors will cause the query to fail diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 67a498cce3dd8..3a771553d3aec 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1221,6 +1221,12 @@ impl SessionConfig { self.options.optimizer.repartition_windows } + /// Are Sorts distributed to multiple cores, If there is already + /// multiple cores in the plan? + pub fn parallelize_sorts(&self) -> bool { + self.options.optimizer.parallelize_sorts + } + /// Are statistics collected during execution? pub fn collect_statistics(&self) -> bool { self.options.execution.collect_statistics @@ -1279,6 +1285,12 @@ impl SessionConfig { self } + /// Enables or disables the use of Sort parallelization + pub fn with_parallelize_sorts(mut self, enabled: bool) -> Self { + self.options.optimizer.parallelize_sorts = enabled; + self + } + /// Enables or disables the use of pruning predicate for parquet readers to skip row groups pub fn with_parquet_pruning(mut self, enabled: bool) -> Self { self.options.execution.parquet.pruning = enabled; @@ -1508,10 +1520,8 @@ impl SessionState { // The EnforceSorting rule is for adding essential local sorting to satisfy the required // ordering. Please make sure that the whole plan tree is determined before this rule. // Note that one should always run this rule after running the EnforceDistribution rule - // as the latter may break local sorting requirements. The rule takes a boolean flag - // indicating whether we elect to transform CoalescePartitionsExec + SortExec cascades - // into SortExec + SortPreservingMergeExec cascades, enabling parallel sorting. - Arc::new(EnforceSorting::new(true)), + // as the latter may break local sorting requirements. + Arc::new(EnforceSorting::new()), // The CoalesceBatches rule will not influence the distribution and ordering of the // whole plan tree. Therefore, to avoid influencing other rules, it should run last. Arc::new(CoalesceBatches::new()), diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs index eaa1b6ab07d51..0dbba2c319b4d 100644 --- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs @@ -1134,7 +1134,7 @@ mod tests { // `EnforceSorting` and `EnfoceDistribution`. // TODO: Orthogonalize the tests here just to verify `EnforceDistribution` and create // new tests for the cascade. - let optimizer = EnforceSorting::new(true); + let optimizer = EnforceSorting::new(); let optimized = optimizer.optimize(optimized, &config)?; // Now format correctly diff --git a/datafusion/core/src/physical_optimizer/repartition.rs b/datafusion/core/src/physical_optimizer/repartition.rs index ef719e1846027..5a3af7bbc068c 100644 --- a/datafusion/core/src/physical_optimizer/repartition.rs +++ b/datafusion/core/src/physical_optimizer/repartition.rs @@ -512,7 +512,7 @@ mod tests { Arc::new(EnforceDistribution::new()), // EnforceSorting is an essential rule to be applied. // Otherwise, the correctness of the generated optimized plan cannot be guaranteed - Arc::new(EnforceSorting::new(true)), + Arc::new(EnforceSorting::new()), ]; let optimized = optimizers.into_iter().fold($PLAN, |plan, optimizer| { optimizer.optimize(plan, &config).unwrap() diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index a4caaf09f07ae..0717e933f098c 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -49,19 +49,14 @@ use std::iter::zip; use std::sync::Arc; /// This rule inspects SortExec's in the given physical plan and removes the -/// ones it can prove unnecessary. The boolean flag `parallelize_sorts` -/// indicates whether we elect to transform CoalescePartitionsExec + SortExec -/// cascades into SortExec + SortPreservingMergeExec cascades, which enables -/// us to perform sorting in parallel. +/// ones it can prove unnecessary. #[derive(Default)] -pub struct EnforceSorting { - parallelize_sorts: bool, -} +pub struct EnforceSorting {} impl EnforceSorting { #[allow(missing_docs)] - pub fn new(parallelize_sorts: bool) -> Self { - Self { parallelize_sorts } + pub fn new() -> Self { + Self {} } } @@ -294,16 +289,20 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { } } +/// The boolean flag `parallelize_sorts` defined in the config +/// indicates whether we elect to transform CoalescePartitionsExec + SortExec +/// cascades into SortExec + SortPreservingMergeExec cascades, which enables +/// us to perform sorting in parallel. impl PhysicalOptimizerRule for EnforceSorting { fn optimize( &self, plan: Arc, - _config: &ConfigOptions, + config: &ConfigOptions, ) -> Result> { // Execute a post-order traversal to adjust input key ordering: let plan_requirements = PlanWithCorrespondingSort::new(plan); let adjusted = plan_requirements.transform_up(&ensure_sorting)?; - if self.parallelize_sorts { + if config.optimizer.parallelize_sorts { let plan_with_coalesce_partitions = PlanWithCorrespondingCoalescePartitions::new(adjusted.plan); let parallel = @@ -1043,7 +1042,7 @@ mod tests { // Run the actual optimizer let optimized_physical_plan = - EnforceSorting::new(true).optimize(physical_plan, state.config_options())?; + EnforceSorting::new().optimize(physical_plan, state.config_options())?; // Get string representation of the plan let actual = get_plan_string(&optimized_physical_plan); assert_eq!( @@ -1736,7 +1735,7 @@ mod tests { let mut plan = orig_plan.clone(); let rules = vec![ Arc::new(EnforceDistribution::new()) as Arc, - Arc::new(EnforceSorting::new(true)) as Arc, + Arc::new(EnforceSorting::new()) as Arc, ]; for rule in rules { plan = rule.optimize(plan, state.config_options())?; @@ -1745,9 +1744,9 @@ mod tests { let mut plan = orig_plan.clone(); let rules = vec![ - Arc::new(EnforceSorting::new(true)) as Arc, + Arc::new(EnforceSorting::new()) as Arc, Arc::new(EnforceDistribution::new()) as Arc, - Arc::new(EnforceSorting::new(true)) as Arc, + Arc::new(EnforceSorting::new()) as Arc, ]; for rule in rules { plan = rule.optimize(plan, state.config_options())?; diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index af24de4f3592d..873ac8d7cf869 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2387,8 +2387,9 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re #[tokio::test] async fn test_window_agg_global_sort() -> Result<()> { let config = SessionConfig::new() - .with_repartition_windows(false) - .with_target_partitions(2); + .with_repartition_windows(true) + .with_target_partitions(2) + .with_parallelize_sorts(true); let ctx = SessionContext::with_config(config); register_aggregate_csv(&ctx).await?; let sql = "SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC"; @@ -2402,9 +2403,50 @@ async fn test_window_agg_global_sort() -> Result<()> { vec![ "SortPreservingMergeExec: [c1@0 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]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", 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)) }]", + " SortExec: [c1@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + ] + }; + + let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual_len = actual.len(); + let actual_trim_last = &actual[..actual_len - 1]; + assert_eq!( + expected, actual_trim_last, + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + Ok(()) +} + +#[tokio::test] +async fn test_window_agg_global_sort_sort_parallelization_disabled() -> Result<()> { + let config = SessionConfig::new() + .with_repartition_windows(true) + .with_target_partitions(2) + .with_parallelize_sorts(false); + let ctx = SessionContext::with_config(config); + register_aggregate_csv(&ctx).await?; + let sql = "SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + let formatted = displayable(physical_plan.as_ref()).indent().to_string(); + // Only 1 SortExec was added + let expected = { + vec![ + "SortExec: [c1@0 ASC NULLS LAST]", + " CoalescePartitionsExec", + " 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(): Ok(Field { name: \"ROW_NUMBER()\", 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)) }]", " SortExec: [c1@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", ] }; diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt index 99b096de1255c..3abe33a551788 100644 --- a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt @@ -129,6 +129,7 @@ datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 datafusion.optimizer.max_passes 3 +datafusion.optimizer.parallelize_sorts true datafusion.optimizer.prefer_hash_join true datafusion.optimizer.repartition_aggregations true datafusion.optimizer.repartition_file_min_size 10485760 @@ -366,4 +367,4 @@ WITH HEADER ROW LOCATION '../../testing/data/csv/aggregate_test_100.csv'; query CCCC SHOW CREATE TABLE abc; ---- -datafusion public abc CREATE EXTERNAL TABLE abc STORED AS CSV LOCATION ../../testing/data/csv/aggregate_test_100.csv \ No newline at end of file +datafusion public abc CREATE EXTERNAL TABLE abc STORED AS CSV LOCATION ../../testing/data/csv/aggregate_test_100.csv diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index cb8478f4483ce..4c56f4e5c2183 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -62,6 +62,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level" | | datafusion.optimizer.repartition_file_scans | false | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | | datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.parallelize_sorts | true | Should DataFusion parallelize Sorts during physical plan creation, if there is already multiple partitions in the plan. | | datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | | datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | From 4d19187926afa855ef504e106035177c31c6a256 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 7 Feb 2023 20:03:42 +0300 Subject: [PATCH 25/31] go back to first commit, add test --- datafusion/core/tests/sql/window.rs | 46 ++++++++++++++++++++++++++++- 1 file changed, 45 insertions(+), 1 deletion(-) diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 873ac8d7cf869..6dcfbe9e73105 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2423,7 +2423,7 @@ async fn test_window_agg_global_sort() -> Result<()> { } #[tokio::test] -async fn test_window_agg_global_sort_sort_parallelization_disabled() -> Result<()> { +async fn test_window_agg_global_sort_parallelize_sort_disabled() -> Result<()> { let config = SessionConfig::new() .with_repartition_windows(true) .with_target_partitions(2) @@ -2461,6 +2461,50 @@ async fn test_window_agg_global_sort_sort_parallelization_disabled() -> Result<( Ok(()) } +#[tokio::test] +async fn test_window_agg_global_sort_intermediate_parallel_sort() -> Result<()> { + let config = SessionConfig::new() + .with_repartition_windows(true) + .with_target_partitions(2) + .with_parallelize_sorts(true); + let ctx = SessionContext::with_config(config); + register_aggregate_csv(&ctx).await?; + let sql = "SELECT c1, \ + SUM(C9) OVER (PARTITION BY C1 ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING) as sum1, \ + SUM(C9) OVER (ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING) as sum2 \ + FROM aggregate_test_100 ORDER BY c1 ASC"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + let formatted = displayable(physical_plan.as_ref()).indent().to_string(); + // Only 1 SortExec was added + let expected = { + vec![ + "SortExec: [c1@0 ASC NULLS LAST]", + " ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2]", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", 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)) }]", + " SortPreservingMergeExec: [c9@1 ASC NULLS LAST]", + " SortExec: [c9@1 ASC NULLS LAST]", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", 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(3)) }]", + " SortExec: [c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + ] + }; + + let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual_len = actual.len(); + let actual_trim_last = &actual[..actual_len - 1]; + assert_eq!( + expected, actual_trim_last, + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + Ok(()) +} + #[tokio::test] async fn test_window_agg_with_global_limit() -> Result<()> { let config = SessionConfig::new() From 663d1631c2b3c8e8293af0616be97656096e0b12 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 7 Feb 2023 20:07:39 +0300 Subject: [PATCH 26/31] update config mg --- docs/source/user-guide/configs.md | 74 +++++++++++++++---------------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 48e1211a2ce48..b0920e09f307a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,40 +35,40 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| --------------------------------------------------------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | false | If the file has a header | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would results in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of cpu cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. EXTRACT(HOUR from SOME_TIME), shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | -| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two read are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | -| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartition to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level" | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level" | -| datafusion.optimizer.repartition_file_scans | false | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level" | -| datafusion.optimizer.parallelize_sorts | true | Should DataFusion parallelize Sorts during physical plan creation, if there is already multiple partitions in the plan. | -| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, sql parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, sql parser will normalize ident(convert ident to lowercase when not quoted) | +| key | default | description | +| --------------------------------------------------------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | false | If the file has a header | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would results in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of cpu cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. EXTRACT(HOUR from SOME_TIME), shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | +| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two read are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | +| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartition to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.repartition_file_scans | false | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.parallelize_sorts | true | Should DataFusion parallelize Sort during physical plan creation. with this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | +| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, sql parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, sql parser will normalize ident(convert ident to lowercase when not quoted) | From d5957ae42eba0d8ca119f0f46ba6a476823e8037 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 7 Feb 2023 16:57:11 -0600 Subject: [PATCH 27/31] Use repartition_sorts in the API instead of parallelize_sorts --- datafusion/common/src/config.rs | 7 ++++--- datafusion/core/src/execution/context.rs | 14 +++++++------- .../src/physical_optimizer/sort_enforcement.rs | 10 +++++----- datafusion/core/tests/sql/window.rs | 6 +++--- .../test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 2 +- 6 files changed, 21 insertions(+), 20 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 6c0f2dafa3457..6ce3f64eeb665 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -290,8 +290,9 @@ config_namespace! { /// functions in parallel using the provided `target_partitions` level" pub repartition_windows: bool, default = true - /// Should DataFusion parallelize Sort during physical plan creation. - /// with this flag is enabled, plans in the form below + /// Should DataFusion execute sorts in a per-partition fashion and merge + /// afterwards instead of coalescing first and sorting globally + /// With this flag is enabled, plans in the form below /// "SortExec: [a@0 ASC]", /// " CoalescePartitionsExec", /// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", @@ -299,7 +300,7 @@ config_namespace! { /// "SortPreservingMergeExec: [a@0 ASC]", /// " SortExec: [a@0 ASC]", /// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - pub parallelize_sorts: bool, default = true + pub repartition_sorts: bool, default = true /// When set to true, the logical plan optimizer will produce warning /// messages if any optimization rules produce errors and then proceed to the next diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 77c3678ab77c6..a053f640fb3bf 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1232,10 +1232,10 @@ impl SessionConfig { self.options.optimizer.repartition_windows } - /// Are Sorts distributed to multiple cores, If there is already - /// multiple cores in the plan? - pub fn parallelize_sorts(&self) -> bool { - self.options.optimizer.parallelize_sorts + /// Do we execute sorts in a per-partition fashion and merge afterwards, + /// or do we coalesce partitions first and sort globally? + pub fn repartition_sorts(&self) -> bool { + self.options.optimizer.repartition_sorts } /// Are statistics collected during execution? @@ -1296,9 +1296,9 @@ impl SessionConfig { self } - /// Enables or disables the use of Sort parallelization - pub fn with_parallelize_sorts(mut self, enabled: bool) -> Self { - self.options.optimizer.parallelize_sorts = enabled; + /// Enables or disables the use of per-partition sorting to improve parallelism + pub fn with_repartition_sorts(mut self, enabled: bool) -> Self { + self.options.optimizer.repartition_sorts = enabled; self } diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 0717e933f098c..c6d93c165d40f 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -289,10 +289,10 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { } } -/// The boolean flag `parallelize_sorts` defined in the config -/// indicates whether we elect to transform CoalescePartitionsExec + SortExec -/// cascades into SortExec + SortPreservingMergeExec cascades, which enables -/// us to perform sorting in parallel. +/// The boolean flag `repartition_sorts` defined in the config indicates +/// whether we elect to transform CoalescePartitionsExec + SortExec cascades +/// into SortExec + SortPreservingMergeExec cascades, which enables us to +/// perform sorting in parallel. impl PhysicalOptimizerRule for EnforceSorting { fn optimize( &self, @@ -302,7 +302,7 @@ impl PhysicalOptimizerRule for EnforceSorting { // Execute a post-order traversal to adjust input key ordering: let plan_requirements = PlanWithCorrespondingSort::new(plan); let adjusted = plan_requirements.transform_up(&ensure_sorting)?; - if config.optimizer.parallelize_sorts { + if config.optimizer.repartition_sorts { let plan_with_coalesce_partitions = PlanWithCorrespondingCoalescePartitions::new(adjusted.plan); let parallel = diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 6dcfbe9e73105..21a6062b8cd7a 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2389,7 +2389,7 @@ async fn test_window_agg_global_sort() -> Result<()> { let config = SessionConfig::new() .with_repartition_windows(true) .with_target_partitions(2) - .with_parallelize_sorts(true); + .with_repartition_sorts(true); let ctx = SessionContext::with_config(config); register_aggregate_csv(&ctx).await?; let sql = "SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC"; @@ -2427,7 +2427,7 @@ async fn test_window_agg_global_sort_parallelize_sort_disabled() -> Result<()> { let config = SessionConfig::new() .with_repartition_windows(true) .with_target_partitions(2) - .with_parallelize_sorts(false); + .with_repartition_sorts(false); let ctx = SessionContext::with_config(config); register_aggregate_csv(&ctx).await?; let sql = "SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC"; @@ -2466,7 +2466,7 @@ async fn test_window_agg_global_sort_intermediate_parallel_sort() -> Result<()> let config = SessionConfig::new() .with_repartition_windows(true) .with_target_partitions(2) - .with_parallelize_sorts(true); + .with_repartition_sorts(true); let ctx = SessionContext::with_config(config); register_aggregate_csv(&ctx).await?; let sql = "SELECT c1, \ diff --git a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt index 60ac7e535793d..75eca2ddff46d 100644 --- a/datafusion/core/tests/sqllogictests/test_files/information_schema.slt +++ b/datafusion/core/tests/sqllogictests/test_files/information_schema.slt @@ -129,12 +129,12 @@ datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 datafusion.optimizer.max_passes 3 -datafusion.optimizer.parallelize_sorts true datafusion.optimizer.prefer_hash_join true datafusion.optimizer.repartition_aggregations true datafusion.optimizer.repartition_file_min_size 10485760 datafusion.optimizer.repartition_file_scans false datafusion.optimizer.repartition_joins true +datafusion.optimizer.repartition_sorts true datafusion.optimizer.repartition_windows true datafusion.optimizer.skip_failed_rules true datafusion.optimizer.top_down_join_key_reordering true diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index b0920e09f307a..7e970c8593536 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -62,7 +62,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level" | | datafusion.optimizer.repartition_file_scans | false | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | | datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level" | -| datafusion.optimizer.parallelize_sorts | true | Should DataFusion parallelize Sort during physical plan creation. with this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion parallelize Sort during physical plan creation. with this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | | datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | | datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | | datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | From 6a63f72ac62d93c2cedfab83a605da886d30c20d Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 7 Feb 2023 17:55:19 -0600 Subject: [PATCH 28/31] Update/format configs.md --- docs/source/user-guide/configs.md | 74 +++++++++++++++---------------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 7e970c8593536..ddfbc4e6e56f3 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,40 +35,40 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| --------------------------------------------------------- | ---------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | false | If the file has a header | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would results in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of cpu cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. EXTRACT(HOUR from SOME_TIME), shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | -| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two read are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | -| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartition to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level" | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level" | -| datafusion.optimizer.repartition_file_scans | false | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level" | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion parallelize Sort during physical plan creation. with this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | -| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, sql parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, sql parser will normalize ident(convert ident to lowercase when not quoted) | +| key | default | description | +| --------------------------------------------------------- | ---------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | false | If the file has a header | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would results in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of cpu cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. EXTRACT(HOUR from SOME_TIME), shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | false | If true, uses parquet data page level metadata (Page Index) statistics to reduce the number of rows decoded. | +| datafusion.execution.parquet.pruning | true | If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two read are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded | +| datafusion.execution.parquet.reorder_filters | false | If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartition to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.repartition_file_scans | false | When set to true, file groups will be repartitioned to achieve maximum parallelism. Currently supported only for Parquet format in which case multiple row groups from the same file may be read concurrently. If false then each row group is read serially, though different files may be read in parallel. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level" | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally With this flag is enabled, plans in the form below "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", would turn into the plan below which performs better in multithreaded environments "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", | +| datafusion.optimizer.skip_failed_rules | true | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, sql parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, sql parser will normalize ident(convert ident to lowercase when not quoted) | From 0ab76da5d8f9be8598cfd2ad0d6953cf2cc9e864 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 8 Feb 2023 13:30:15 +0300 Subject: [PATCH 29/31] Use maintains_input_order instead of output_ordering comparison --- .../physical_optimizer/sort_enforcement.rs | 37 ++++++++++--------- 1 file changed, 20 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index c6d93c165d40f..18a731a305ce0 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -38,6 +38,7 @@ use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::rewrite::TreeNodeRewritable; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; use arrow::datatypes::SchemaRef; @@ -150,24 +151,26 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { return None; } let is_spm = is_sort_preserving_merge(plan); - let output_ordering = plan.output_ordering(); + let is_union = plan.as_any().is::(); + // If the executor is a `UnionExec`, and it has an output ordering; + // then it at least partially maintains some child's output ordering. + // Therefore, we propagate this information upwards. + let partially_maintains = + is_union && plan.output_ordering().is_some(); let required_orderings = plan.required_input_ordering(); - let children = - izip!(&plan.children(), item.sort_onwards, required_orderings) - .filter_map(|(child, element, required_ordering)| { - // Executor maintains or partially maintains its child's output ordering - let maintains = ordering_satisfy( - child.output_ordering(), - output_ordering, - || child.equivalence_properties(), - ); - if (required_ordering.is_none() && maintains) || is_spm { - element - } else { - None - } - }) - .collect::>(); + let flags = plan.maintains_input_order(); + let children = izip!(flags, item.sort_onwards, required_orderings) + .filter_map(|(maintains, element, required_ordering)| { + if (required_ordering.is_none() + && (maintains || partially_maintains)) + || is_spm + { + element + } else { + None + } + }) + .collect::>(); if !children.is_empty() { // Add parent node to the tree if there is at least one // child with a subtree: From 03c4fbe1006fbf8997ff2e94f76410213d22d752 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 8 Feb 2023 13:39:01 +0300 Subject: [PATCH 30/31] Move logic under map_children to init method --- .../physical_optimizer/sort_enforcement.rs | 254 ++++++++++-------- 1 file changed, 137 insertions(+), 117 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 18a731a305ce0..653d3567261c4 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -105,6 +105,73 @@ impl PlanWithCorrespondingSort { } } + pub fn new_from_children_nodes( + children_nodes: Vec, + parent_plan: Arc, + ) -> Result { + let children_plans = children_nodes + .iter() + .map(|item| item.plan.clone()) + .collect::>(); + + let sort_onwards = children_nodes + .into_iter() + .enumerate() + .map(|(idx, item)| { + let plan = &item.plan; + // Leaves of the `sort_onwards` are `SortExec`(Introduces ordering). This tree collects + // all the intermediate executors that maintain this ordering. If + // we just saw a sort-introducing operator, we reset the tree and + // start accumulating. + if is_sort(plan) { + return Some(ExecTree { + idx, + plan: item.plan, + children: vec![], + }); + } else if is_limit(plan) { + // There is no sort linkage for this path, it starts at a limit. + return None; + } + let is_spm = is_sort_preserving_merge(plan); + let is_union = plan.as_any().is::(); + // If the executor is a `UnionExec`, and it has an output ordering; + // then it at least partially maintains some child's output ordering. + // Therefore, we propagate this information upwards. + let partially_maintains = is_union && plan.output_ordering().is_some(); + let required_orderings = plan.required_input_ordering(); + let flags = plan.maintains_input_order(); + let children = izip!(flags, item.sort_onwards, required_orderings) + .filter_map(|(maintains, element, required_ordering)| { + if (required_ordering.is_none() + && (maintains || partially_maintains)) + || is_spm + { + element + } else { + None + } + }) + .collect::>(); + if !children.is_empty() { + // Add parent node to the tree if there is at least one + // child with a subtree: + Some(ExecTree { + idx, + plan: item.plan, + children, + }) + } else { + // There is no sort linkage for this child, do nothing. + None + } + }) + .collect(); + + let plan = with_new_children_if_necessary(parent_plan, children_plans)?; + Ok(PlanWithCorrespondingSort { plan, sort_onwards }) + } + pub fn children(&self) -> Vec { self.plan .children() @@ -123,70 +190,14 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { if children.is_empty() { Ok(self) } else { - let children_requirements = children + let children_nodes = children .into_iter() .map(transform) .collect::>>()?; - let children_plans = children_requirements - .iter() - .map(|item| item.plan.clone()) - .collect::>(); - let sort_onwards = children_requirements - .into_iter() - .enumerate() - .map(|(idx, item)| { - let plan = &item.plan; - // Leaves of the `sort_onwards` are `SortExec`(Introduces ordering). This tree collects - // all the intermediate executors that maintain this ordering. If - // we just saw a sort-introducing operator, we reset the tree and - // start accumulating. - if is_sort(plan) { - return Some(ExecTree { - idx, - plan: item.plan, - children: vec![], - }); - } else if is_limit(plan) { - // There is no sort linkage for this path, it starts at a limit. - return None; - } - let is_spm = is_sort_preserving_merge(plan); - let is_union = plan.as_any().is::(); - // If the executor is a `UnionExec`, and it has an output ordering; - // then it at least partially maintains some child's output ordering. - // Therefore, we propagate this information upwards. - let partially_maintains = - is_union && plan.output_ordering().is_some(); - let required_orderings = plan.required_input_ordering(); - let flags = plan.maintains_input_order(); - let children = izip!(flags, item.sort_onwards, required_orderings) - .filter_map(|(maintains, element, required_ordering)| { - if (required_ordering.is_none() - && (maintains || partially_maintains)) - || is_spm - { - element - } else { - None - } - }) - .collect::>(); - if !children.is_empty() { - // Add parent node to the tree if there is at least one - // child with a subtree: - Some(ExecTree { - idx, - plan: item.plan, - children, - }) - } else { - // There is no sort linkage for this child, do nothing. - None - } - }) - .collect(); - let plan = with_new_children_if_necessary(self.plan, children_plans)?; - Ok(PlanWithCorrespondingSort { plan, sort_onwards }) + PlanWithCorrespondingSort::new_from_children_nodes( + children_nodes, + self.plan.clone(), + ) } } } @@ -200,7 +211,7 @@ struct PlanWithCorrespondingCoalescePartitions { // child until the `CoalescePartitionsExec`(s) -- could be multiple for // n-ary plans like Union -- that affect the output partitioning of the // child. If the child has no connection to any `CoalescePartitionsExec`, - // simpliy store None (and not a subtree). + // simplify store None (and not a subtree). coalesce_onwards: Vec>, } @@ -213,6 +224,65 @@ impl PlanWithCorrespondingCoalescePartitions { } } + pub fn new_from_children_nodes( + children_nodes: Vec, + parent_plan: Arc, + ) -> Result { + let children_plans = children_nodes + .iter() + .map(|item| item.plan.clone()) + .collect(); + let coalesce_onwards = children_nodes + .into_iter() + .enumerate() + .map(|(idx, item)| { + // Leaves of the `coalesce_onwards` tree are `CoalescePartitionsExec` + // operators. This tree collects all the intermediate executors that + // maintain a single partition. If we just saw a `CoalescePartitionsExec` + // operator, we reset the tree and start accumulating. + let plan = item.plan; + if plan.as_any().is::() { + Some(ExecTree { + idx, + plan, + children: vec![], + }) + } else if plan.children().is_empty() { + // Plan has no children, there is nothing to propagate. + None + } else { + let children = item + .coalesce_onwards + .into_iter() + .flatten() + .filter(|item| { + // Only consider operators that don't require a + // single partition. + !matches!( + plan.required_input_distribution()[item.idx], + Distribution::SinglePartition + ) + }) + .collect::>(); + if children.is_empty() { + None + } else { + Some(ExecTree { + idx, + plan, + children, + }) + } + } + }) + .collect(); + let plan = with_new_children_if_necessary(parent_plan, children_plans)?; + Ok(PlanWithCorrespondingCoalescePartitions { + plan, + coalesce_onwards, + }) + } + pub fn children(&self) -> Vec { self.plan .children() @@ -231,63 +301,14 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { if children.is_empty() { Ok(self) } else { - let children_requirements = children + let children_nodes = children .into_iter() .map(transform) .collect::>>()?; - let children_plans = children_requirements - .iter() - .map(|item| item.plan.clone()) - .collect(); - let coalesce_onwards = children_requirements - .into_iter() - .enumerate() - .map(|(idx, item)| { - // Leaves of the `coalesce_onwards` tree are `CoalescePartitionsExec` - // operators. This tree collects all the intermediate executors that - // maintain a single partition. If we just saw a `CoalescePartitionsExec` - // operator, we reset the tree and start accumulating. - let plan = item.plan; - if plan.as_any().is::() { - Some(ExecTree { - idx, - plan, - children: vec![], - }) - } else if plan.children().is_empty() { - // Plan has no children, there is nothing to propagate. - None - } else { - let children = item - .coalesce_onwards - .into_iter() - .flatten() - .filter(|item| { - // Only consider operators that don't require a - // single partition. - !matches!( - plan.required_input_distribution()[item.idx], - Distribution::SinglePartition - ) - }) - .collect::>(); - if children.is_empty() { - None - } else { - Some(ExecTree { - idx, - plan, - children, - }) - } - } - }) - .collect(); - let plan = with_new_children_if_necessary(self.plan, children_plans)?; - Ok(PlanWithCorrespondingCoalescePartitions { - plan, - coalesce_onwards, - }) + PlanWithCorrespondingCoalescePartitions::new_from_children_nodes( + children_nodes, + self.plan.clone(), + ) } } } @@ -302,7 +323,6 @@ impl PhysicalOptimizerRule for EnforceSorting { plan: Arc, config: &ConfigOptions, ) -> Result> { - // Execute a post-order traversal to adjust input key ordering: let plan_requirements = PlanWithCorrespondingSort::new(plan); let adjusted = plan_requirements.transform_up(&ensure_sorting)?; if config.optimizer.repartition_sorts { From a25ed96ce75c8c5cbdf4e3e0f9fec3c24cc4af43 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 8 Feb 2023 13:40:57 -0600 Subject: [PATCH 31/31] Remove two unnecessary clones --- .../src/physical_optimizer/sort_enforcement.rs | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 653d3567261c4..0eadfb7c2dea5 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -113,16 +113,15 @@ impl PlanWithCorrespondingSort { .iter() .map(|item| item.plan.clone()) .collect::>(); - let sort_onwards = children_nodes .into_iter() .enumerate() .map(|(idx, item)| { let plan = &item.plan; - // Leaves of the `sort_onwards` are `SortExec`(Introduces ordering). This tree collects - // all the intermediate executors that maintain this ordering. If - // we just saw a sort-introducing operator, we reset the tree and - // start accumulating. + // Leaves of `sort_onwards` are `SortExec` operators, which impose + // an ordering. This tree collects all the intermediate executors + // that maintain this ordering. If we just saw a order imposing + // operator, we reset the tree and start accumulating. if is_sort(plan) { return Some(ExecTree { idx, @@ -194,10 +193,7 @@ impl TreeNodeRewritable for PlanWithCorrespondingSort { .into_iter() .map(transform) .collect::>>()?; - PlanWithCorrespondingSort::new_from_children_nodes( - children_nodes, - self.plan.clone(), - ) + PlanWithCorrespondingSort::new_from_children_nodes(children_nodes, self.plan) } } } @@ -307,7 +303,7 @@ impl TreeNodeRewritable for PlanWithCorrespondingCoalescePartitions { .collect::>>()?; PlanWithCorrespondingCoalescePartitions::new_from_children_nodes( children_nodes, - self.plan.clone(), + self.plan, ) } }