From 510b16c8f4aaaf434ee63e94d7215cfe578afba3 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 09:17:29 +0300 Subject: [PATCH 01/30] Tmp --- .../src/physical_optimizer/enforce_sorting.rs | 65 +++++++++++++++++-- .../src/physical_optimizer/sort_pushdown.rs | 29 ++++++++- datafusion/sqllogictest/test_files/window.slt | 22 +++---- 3 files changed, 97 insertions(+), 19 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index faf8d01a97fd..c5785f7127be 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -64,7 +64,8 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; -use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::{displayable, ExecutionPlanProperties}; +use datafusion_physical_expr::Partitioning; use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::izip; @@ -148,6 +149,13 @@ fn update_coalesce_ctx_children( }; } +fn print_plan(plan: &Arc) { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + + /// 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 @@ -162,6 +170,10 @@ impl PhysicalOptimizerRule for EnforceSorting { // Execute a bottom-up traversal to enforce sorting requirements, // remove unnecessary sorts, and optimize sort-sensitive operators: let adjusted = plan_requirements.transform_up(ensure_sorting)?.data; + if false { + println!("After ensure sorting"); + print_plan(&adjusted.plan); + } let new_plan = if config.optimizer.repartition_sorts { let plan_with_coalesce_partitions = PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan); @@ -172,7 +184,10 @@ impl PhysicalOptimizerRule for EnforceSorting { } else { adjusted.plan }; - + if false { + println!("After parallelize_sorts"); + print_plan(&new_plan); + } let plan_with_pipeline_fixer = OrderPreservationContext::new_default(new_plan); let updated_plan = plan_with_pipeline_fixer .transform_up(|plan_with_pipeline_fixer| { @@ -513,8 +528,20 @@ fn remove_corresponding_coalesce_in_sub_plan( }) .collect::>()?; } - - requirements.update_plan_from_children() + let mut new_req = requirements.update_plan_from_children()?; + if let Some(repartition) = new_req.plan.as_any().downcast_ref::() { + let mut can_remove = false; + if repartition.input().output_partitioning().eq(repartition.partitioning()) { + // Their partitioning same + can_remove = true; + } else if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning(){ + can_remove = *n_out == repartition.input().output_partitioning().partition_count(); + } + if can_remove { + new_req = new_req.children.swap_remove(0) + } + } + Ok(new_req) } /// Updates child to remove the unnecessary sort below it. @@ -540,8 +567,12 @@ fn remove_corresponding_sort_from_sub_plan( requires_single_partition: bool, ) -> Result { // A `SortExec` is always at the bottom of the tree. - if is_sort(&node.plan) { - node = node.children.swap_remove(0); + if let Some(sort_exec) = node.plan.as_any().downcast_ref::(){ + if sort_exec.fetch().is_none() { + node = node.children.swap_remove(0); + } else { + // Do not remove the sort with fetch + } } else { let mut any_connection = false; let required_dist = node.plan.required_input_distribution(); @@ -1049,6 +1080,28 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_remove_unnecessary_sort6() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let input = Arc::new(SortExec::new(vec![sort_expr("non_nullable_col", &schema)], source).with_fetch(Some(2))); + let physical_plan = sort_exec(vec![sort_expr("non_nullable_col", &schema), sort_expr("nullable_col", &schema)], input); + + let expected_input = [ + "SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + #[tokio::test] async fn test_remove_unnecessary_spm1() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 3577e109b069..af597cbdc1c1 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -37,6 +37,7 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_plan::displayable; /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total @@ -54,6 +55,12 @@ pub fn assign_initial_requirements(node: &mut SortPushDown) { } } +fn print_plan(plan: &Arc) { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + pub(crate) fn pushdown_sorts( mut requirements: SortPushDown, ) -> Result> { @@ -64,11 +71,13 @@ pub(crate) fn pushdown_sorts( .ordering_satisfy_requirement(parent_reqs); if let Some(sort_exec) = plan.as_any().downcast_ref::() { + println!("sort operator"); + print_plan(&plan); let required_ordering = plan .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs) .unwrap_or_default(); - + println!("required_ordering: {:?}", required_ordering); if !satisfy_parent { // Make sure this `SortExec` satisfies parent requirements: let fetch = sort_exec.fetch(); @@ -85,9 +94,13 @@ pub(crate) fn pushdown_sorts( for (grand_child, order) in child.children.iter_mut().zip(adjusted) { grand_child.data = order; } + println!("pushed down requirements plan"); + print_plan(&requirements.plan); + println!("pushed down child plan"); + print_plan(&child.plan); // Can push down requirements child.data = None; - return Ok(Transformed::yes(child)); + requirements = child; } else { // Can not push down requirements requirements.children = vec![child]; @@ -132,6 +145,18 @@ fn pushdown_requirement_to_children( RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), RequirementsCompatibility::NonCompatible => Ok(None), } + } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { + println!("trying to pushdown sort exec"); + println!("parent_required: {:?}", parent_required); + println!("sort_exec.properties().eq_properties: {:?}", sort_exec.properties().eq_properties); + let sort_req = PhysicalSortRequirement::from_sort_exprs(sort_exec.properties().output_ordering().unwrap_or(&[])); + if sort_exec.properties().eq_properties.requirements_compatible(parent_required, &sort_req){ + println!("compatible, parent_required: {:?}", parent_required); + debug_assert!(!parent_required.is_empty()); + Ok(Some(vec![Some(parent_required.to_vec())])) + } else { + Ok(None) + } } else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 4f4b9749c561..dfc882667617 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1777,17 +1777,17 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[count(*)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(*)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=2 -06)----------ProjectionExec: expr=[] -07)------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] -08)--------------CoalesceBatchesExec: target_batch_size=4096 -09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -10)------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -11)--------------------ProjectionExec: expr=[c1@0 as c1] -12)----------------------CoalesceBatchesExec: target_batch_size=4096 -13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 -14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +05)--------ProjectionExec: expr=[] +06)----------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] +07)------------CoalesceBatchesExec: target_batch_size=4096 +08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] +10)------------------ProjectionExec: expr=[c1@0 as c1] +11)--------------------CoalesceBatchesExec: target_batch_size=4096 +12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true + query I SELECT count(*) as global_count FROM From 6ef43693b4ab14a83e1d5b746ab12003dc302087 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 10:29:45 +0300 Subject: [PATCH 02/30] Minor changes --- .../src/physical_optimizer/enforce_sorting.rs | 38 ++++++-- .../src/physical_optimizer/sort_pushdown.rs | 88 ++++++++++++++----- 2 files changed, 96 insertions(+), 30 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index c5785f7127be..db8b260c12e9 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -43,9 +43,7 @@ use crate::error::Result; use crate::physical_optimizer::replace_with_order_preserving_variants::{ replace_with_order_preserving_variants, OrderPreservationContext, }; -use crate::physical_optimizer::sort_pushdown::{ - assign_initial_requirements, pushdown_sorts, SortPushDown, -}; +use crate::physical_optimizer::sort_pushdown::{assign_initial_requirements, prune_unnecessary_operators, pushdown_sorts, SortPushDown}; use crate::physical_optimizer::utils::{ is_coalesce_partitions, is_limit, is_repartition, is_sort, is_sort_preserving_merge, is_union, is_window, @@ -205,11 +203,12 @@ impl PhysicalOptimizerRule for EnforceSorting { let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); assign_initial_requirements(&mut sort_pushdown); let adjusted = sort_pushdown.transform_down(pushdown_sorts)?.data; - - adjusted + let plan = adjusted .plan .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?))) - .data() + .data()?; + // Prune out unnecessary operators from the plan. + prune_unnecessary_operators(plan) } fn name(&self) -> &str { @@ -666,6 +665,7 @@ mod tests { use datafusion_physical_optimizer::PhysicalOptimizerRule; use rstest::rstest; + use datafusion_physical_plan::limit::GlobalLimitExec; fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); @@ -1102,6 +1102,32 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_do_not_pushdown_through_limit() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let input = Arc::new(SortExec::new(vec![sort_expr("non_nullable_col", &schema)], source)); + let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; + let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], limit); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " GlobalLimitExec: skip=0, fetch=5", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " GlobalLimitExec: skip=0, fetch=5", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + #[tokio::test] async fn test_remove_unnecessary_spm1() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index af597cbdc1c1..63d7417d0b9b 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -30,7 +30,7 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::tree_node::PlanContext; use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; -use datafusion_common::tree_node::Transformed; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; @@ -39,19 +39,25 @@ use datafusion_physical_expr::{ }; use datafusion_physical_plan::displayable; +#[derive(Default, Clone)] +pub struct ParentRequirements{ + ordering_requirement: Option>, + fetch: Option, +} + /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total /// computational cost by pushing down `SortExec`s through some executors. The /// object carries the parent required ordering as its data. /// /// [`EnforceSorting`]: crate::physical_optimizer::enforce_sorting::EnforceSorting -pub type SortPushDown = PlanContext>>; +pub type SortPushDown = PlanContext; /// Assigns the ordering requirement of the root node to the its children. pub fn assign_initial_requirements(node: &mut SortPushDown) { let reqs = node.plan.required_input_ordering(); for (child, requirement) in node.children.iter_mut().zip(reqs) { - child.data = requirement; + child.data = ParentRequirements{ordering_requirement: requirement, fetch: None}; } } @@ -61,27 +67,60 @@ fn print_plan(plan: &Arc) { println!("{:#?}", actual); } +/// Get fetch information for the operator. +fn get_fetch(plan: &Arc) -> Option{ + if let Some(sort) =plan.as_any().downcast_ref::(){ + sort.fetch() + } else { + None + } +} + + +/// Prunes unnecessary operators from the plan. +pub(crate) fn prune_unnecessary_operators(plan: Arc) -> Result>{ + Ok(plan.transform_up(|p| { + if let Some(sort) = p.as_any().downcast_ref::(){ + let out_ordering = sort.expr(); + if sort.input().equivalence_properties().ordering_satisfy(out_ordering) { + // Ordering is satisfied + let out_fetch = sort.fetch(); + let in_fetch = get_fetch(sort.input()); + let can_remove = match (out_fetch, in_fetch) { + (Some(out_fetch), Some(in_fetch)) => { + in_fetch >= out_fetch + }, + (None, _) => true, + // Cannot remove, as sort decreases number of rows after processing. + (Some(_), None) => false, + }; + if can_remove{ + return Ok(Transformed::yes(sort.input().clone())) + } + } + } + Ok(Transformed::no(p)) + })?.data) +} + pub(crate) fn pushdown_sorts( mut requirements: SortPushDown, ) -> Result> { let plan = &requirements.plan; - let parent_reqs = requirements.data.as_deref().unwrap_or(&[]); + let parent_reqs = requirements.data.ordering_requirement.as_deref().unwrap_or(&[]); let satisfy_parent = plan .equivalence_properties() .ordering_satisfy_requirement(parent_reqs); if let Some(sort_exec) = plan.as_any().downcast_ref::() { - println!("sort operator"); - print_plan(&plan); let required_ordering = plan .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs) .unwrap_or_default(); - println!("required_ordering: {:?}", required_ordering); if !satisfy_parent { // Make sure this `SortExec` satisfies parent requirements: let fetch = sort_exec.fetch(); - let sort_reqs = requirements.data.unwrap_or_default(); + let sort_reqs = requirements.data.ordering_requirement.unwrap_or_default(); requirements = requirements.children.swap_remove(0); requirements = add_sort_above(requirements, sort_reqs, fetch); }; @@ -91,16 +130,14 @@ pub(crate) fn pushdown_sorts( if let Some(adjusted) = pushdown_requirement_to_children(&child.plan, &required_ordering)? { + let fetch = get_fetch(&child.plan); for (grand_child, order) in child.children.iter_mut().zip(adjusted) { - grand_child.data = order; + grand_child.data = ParentRequirements{ordering_requirement: order, fetch}; } - println!("pushed down requirements plan"); - print_plan(&requirements.plan); - println!("pushed down child plan"); - print_plan(&child.plan); // Can push down requirements - child.data = None; - requirements = child; + child.data = ParentRequirements{ordering_requirement: None, fetch}; + print_plan(&child.plan); + return Ok(Transformed::yes(child)); } else { // Can not push down requirements requirements.children = vec![child]; @@ -110,19 +147,20 @@ pub(crate) fn pushdown_sorts( // For non-sort operators, immediately return if parent requirements are met: let reqs = plan.required_input_ordering(); for (child, order) in requirements.children.iter_mut().zip(reqs) { - child.data = order; + child.data.ordering_requirement = order; } } else if let Some(adjusted) = pushdown_requirement_to_children(plan, parent_reqs)? { // Can not satisfy the parent requirements, check whether we can push // requirements down: for (child, order) in requirements.children.iter_mut().zip(adjusted) { - child.data = order; + child.data.ordering_requirement = order; } - requirements.data = None; + requirements.data.ordering_requirement = None; } else { // Can not push down requirements, add new `SortExec`: - let sort_reqs = requirements.data.clone().unwrap_or_default(); - requirements = add_sort_above(requirements, sort_reqs, None); + let sort_reqs = requirements.data.ordering_requirement.clone().unwrap_or_default(); + let fetch = requirements.data.fetch; + requirements = add_sort_above(requirements, sort_reqs, fetch); assign_initial_requirements(&mut requirements); } Ok(Transformed::yes(requirements)) @@ -145,10 +183,11 @@ fn pushdown_requirement_to_children( RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), RequirementsCompatibility::NonCompatible => Ok(None), } - } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { + } + else if let Some(sort_exec) = plan.as_any().downcast_ref::() { println!("trying to pushdown sort exec"); - println!("parent_required: {:?}", parent_required); - println!("sort_exec.properties().eq_properties: {:?}", sort_exec.properties().eq_properties); + // println!("parent_required: {:?}", parent_required); + // println!("sort_exec.properties().eq_properties: {:?}", sort_exec.properties().eq_properties); let sort_req = PhysicalSortRequirement::from_sort_exprs(sort_exec.properties().output_ordering().unwrap_or(&[])); if sort_exec.properties().eq_properties.requirements_compatible(parent_required, &sort_req){ println!("compatible, parent_required: {:?}", parent_required); @@ -157,7 +196,8 @@ fn pushdown_requirement_to_children( } else { Ok(None) } - } else if is_union(plan) { + } + else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec let req = (!parent_required.is_empty()).then(|| parent_required.to_vec()); From c3efafc0ea136bbdd138b38f1398e3842de77d5b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 10:32:17 +0300 Subject: [PATCH 03/30] Minor changes --- .../src/physical_optimizer/enforce_sorting.rs | 59 +++++----- .../src/physical_optimizer/sort_pushdown.rs | 106 ++++++++++-------- 2 files changed, 94 insertions(+), 71 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index db8b260c12e9..f857c1cc83e7 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -43,7 +43,10 @@ use crate::error::Result; use crate::physical_optimizer::replace_with_order_preserving_variants::{ replace_with_order_preserving_variants, OrderPreservationContext, }; -use crate::physical_optimizer::sort_pushdown::{assign_initial_requirements, prune_unnecessary_operators, pushdown_sorts, SortPushDown}; +use crate::physical_optimizer::sort_pushdown::{ + assign_initial_requirements, prune_unnecessary_operators, pushdown_sorts, + SortPushDown, +}; use crate::physical_optimizer::utils::{ is_coalesce_partitions, is_limit, is_repartition, is_sort, is_sort_preserving_merge, is_union, is_window, @@ -59,11 +62,11 @@ use crate::physical_plan::{Distribution, ExecutionPlan, InputOrderMode}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_physical_expr::Partitioning; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; -use datafusion_physical_plan::{displayable, ExecutionPlanProperties}; -use datafusion_physical_expr::Partitioning; +use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::izip; @@ -147,13 +150,6 @@ fn update_coalesce_ctx_children( }; } -fn print_plan(plan: &Arc) { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - - /// 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 @@ -168,10 +164,6 @@ impl PhysicalOptimizerRule for EnforceSorting { // Execute a bottom-up traversal to enforce sorting requirements, // remove unnecessary sorts, and optimize sort-sensitive operators: let adjusted = plan_requirements.transform_up(ensure_sorting)?.data; - if false { - println!("After ensure sorting"); - print_plan(&adjusted.plan); - } let new_plan = if config.optimizer.repartition_sorts { let plan_with_coalesce_partitions = PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan); @@ -182,10 +174,6 @@ impl PhysicalOptimizerRule for EnforceSorting { } else { adjusted.plan }; - if false { - println!("After parallelize_sorts"); - print_plan(&new_plan); - } let plan_with_pipeline_fixer = OrderPreservationContext::new_default(new_plan); let updated_plan = plan_with_pipeline_fixer .transform_up(|plan_with_pipeline_fixer| { @@ -530,11 +518,16 @@ fn remove_corresponding_coalesce_in_sub_plan( let mut new_req = requirements.update_plan_from_children()?; if let Some(repartition) = new_req.plan.as_any().downcast_ref::() { let mut can_remove = false; - if repartition.input().output_partitioning().eq(repartition.partitioning()) { + if repartition + .input() + .output_partitioning() + .eq(repartition.partitioning()) + { // Their partitioning same can_remove = true; - } else if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning(){ - can_remove = *n_out == repartition.input().output_partitioning().partition_count(); + } else if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() { + can_remove = + *n_out == repartition.input().output_partitioning().partition_count(); } if can_remove { new_req = new_req.children.swap_remove(0) @@ -566,7 +559,7 @@ fn remove_corresponding_sort_from_sub_plan( requires_single_partition: bool, ) -> Result { // A `SortExec` is always at the bottom of the tree. - if let Some(sort_exec) = node.plan.as_any().downcast_ref::(){ + if let Some(sort_exec) = node.plan.as_any().downcast_ref::() { if sort_exec.fetch().is_none() { node = node.children.swap_remove(0); } else { @@ -662,11 +655,11 @@ mod tests { use datafusion_common::Result; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::{col, Column, NotExpr}; - use datafusion_physical_optimizer::PhysicalOptimizerRule; - use rstest::rstest; use datafusion_physical_plan::limit::GlobalLimitExec; + use rstest::rstest; + fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); @@ -1085,8 +1078,17 @@ mod tests { let schema = create_test_schema()?; let source = memory_exec(&schema); // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input = Arc::new(SortExec::new(vec![sort_expr("non_nullable_col", &schema)], source).with_fetch(Some(2))); - let physical_plan = sort_exec(vec![sort_expr("non_nullable_col", &schema), sort_expr("nullable_col", &schema)], input); + let input = Arc::new( + SortExec::new(vec![sort_expr("non_nullable_col", &schema)], source) + .with_fetch(Some(2)), + ); + let physical_plan = sort_exec( + vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ], + input, + ); let expected_input = [ "SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", @@ -1107,7 +1109,10 @@ mod tests { let schema = create_test_schema()?; let source = memory_exec(&schema); // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input = Arc::new(SortExec::new(vec![sort_expr("non_nullable_col", &schema)], source)); + let input = Arc::new(SortExec::new( + vec![sort_expr("non_nullable_col", &schema)], + source, + )); let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], limit); diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 63d7417d0b9b..b8ca5fc6979a 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -37,10 +37,9 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; -use datafusion_physical_plan::displayable; #[derive(Default, Clone)] -pub struct ParentRequirements{ +pub struct ParentRequirements { ordering_requirement: Option>, fetch: Option, } @@ -57,57 +56,63 @@ pub type SortPushDown = PlanContext; pub fn assign_initial_requirements(node: &mut SortPushDown) { let reqs = node.plan.required_input_ordering(); for (child, requirement) in node.children.iter_mut().zip(reqs) { - child.data = ParentRequirements{ordering_requirement: requirement, fetch: None}; + child.data = ParentRequirements { + ordering_requirement: requirement, + fetch: None, + }; } } -fn print_plan(plan: &Arc) { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - /// Get fetch information for the operator. -fn get_fetch(plan: &Arc) -> Option{ - if let Some(sort) =plan.as_any().downcast_ref::(){ +fn get_fetch(plan: &Arc) -> Option { + if let Some(sort) = plan.as_any().downcast_ref::() { sort.fetch() } else { None } } - /// Prunes unnecessary operators from the plan. -pub(crate) fn prune_unnecessary_operators(plan: Arc) -> Result>{ - Ok(plan.transform_up(|p| { - if let Some(sort) = p.as_any().downcast_ref::(){ - let out_ordering = sort.expr(); - if sort.input().equivalence_properties().ordering_satisfy(out_ordering) { - // Ordering is satisfied - let out_fetch = sort.fetch(); - let in_fetch = get_fetch(sort.input()); - let can_remove = match (out_fetch, in_fetch) { - (Some(out_fetch), Some(in_fetch)) => { - in_fetch >= out_fetch - }, - (None, _) => true, - // Cannot remove, as sort decreases number of rows after processing. - (Some(_), None) => false, - }; - if can_remove{ - return Ok(Transformed::yes(sort.input().clone())) +pub(crate) fn prune_unnecessary_operators( + plan: Arc, +) -> Result> { + Ok(plan + .transform_up(|p| { + if let Some(sort) = p.as_any().downcast_ref::() { + let out_ordering = sort.expr(); + if sort + .input() + .equivalence_properties() + .ordering_satisfy(out_ordering) + { + // Ordering is satisfied + let out_fetch = sort.fetch(); + let in_fetch = get_fetch(sort.input()); + let can_remove = match (out_fetch, in_fetch) { + (Some(out_fetch), Some(in_fetch)) => in_fetch >= out_fetch, + (None, _) => true, + // Cannot remove, as sort decreases number of rows after processing. + (Some(_), None) => false, + }; + if can_remove { + return Ok(Transformed::yes(sort.input().clone())); + } } } - } - Ok(Transformed::no(p)) - })?.data) + Ok(Transformed::no(p)) + })? + .data) } pub(crate) fn pushdown_sorts( mut requirements: SortPushDown, ) -> Result> { let plan = &requirements.plan; - let parent_reqs = requirements.data.ordering_requirement.as_deref().unwrap_or(&[]); + let parent_reqs = requirements + .data + .ordering_requirement + .as_deref() + .unwrap_or(&[]); let satisfy_parent = plan .equivalence_properties() .ordering_satisfy_requirement(parent_reqs); @@ -132,11 +137,16 @@ pub(crate) fn pushdown_sorts( { let fetch = get_fetch(&child.plan); for (grand_child, order) in child.children.iter_mut().zip(adjusted) { - grand_child.data = ParentRequirements{ordering_requirement: order, fetch}; + grand_child.data = ParentRequirements { + ordering_requirement: order, + fetch, + }; } // Can push down requirements - child.data = ParentRequirements{ordering_requirement: None, fetch}; - print_plan(&child.plan); + child.data = ParentRequirements { + ordering_requirement: None, + fetch, + }; return Ok(Transformed::yes(child)); } else { // Can not push down requirements @@ -158,7 +168,11 @@ pub(crate) fn pushdown_sorts( requirements.data.ordering_requirement = None; } else { // Can not push down requirements, add new `SortExec`: - let sort_reqs = requirements.data.ordering_requirement.clone().unwrap_or_default(); + let sort_reqs = requirements + .data + .ordering_requirement + .clone() + .unwrap_or_default(); let fetch = requirements.data.fetch; requirements = add_sort_above(requirements, sort_reqs, fetch); assign_initial_requirements(&mut requirements); @@ -183,21 +197,25 @@ fn pushdown_requirement_to_children( RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), RequirementsCompatibility::NonCompatible => Ok(None), } - } - else if let Some(sort_exec) = plan.as_any().downcast_ref::() { + } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { println!("trying to pushdown sort exec"); // println!("parent_required: {:?}", parent_required); // println!("sort_exec.properties().eq_properties: {:?}", sort_exec.properties().eq_properties); - let sort_req = PhysicalSortRequirement::from_sort_exprs(sort_exec.properties().output_ordering().unwrap_or(&[])); - if sort_exec.properties().eq_properties.requirements_compatible(parent_required, &sort_req){ + let sort_req = PhysicalSortRequirement::from_sort_exprs( + sort_exec.properties().output_ordering().unwrap_or(&[]), + ); + if sort_exec + .properties() + .eq_properties + .requirements_compatible(parent_required, &sort_req) + { println!("compatible, parent_required: {:?}", parent_required); debug_assert!(!parent_required.is_empty()); Ok(Some(vec![Some(parent_required.to_vec())])) } else { Ok(None) } - } - else if is_union(plan) { + } else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec let req = (!parent_required.is_empty()).then(|| parent_required.to_vec()); From 2bf220d9b67034f34d3d9f21cefa9e41bae1324b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 14:09:25 +0300 Subject: [PATCH 04/30] Minor changes --- .../src/physical_optimizer/enforce_sorting.rs | 36 ++++- .../src/physical_optimizer/sort_pushdown.rs | 135 ++++++++++++------ .../physical-plan/src/coalesce_batches.rs | 4 + .../physical-plan/src/execution_plan.rs | 5 + datafusion/physical-plan/src/limit.rs | 8 ++ datafusion/physical-plan/src/sorts/sort.rs | 4 + 6 files changed, 150 insertions(+), 42 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index f857c1cc83e7..56ce7b70aaaa 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -656,7 +656,7 @@ mod tests { use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::{col, Column, NotExpr}; use datafusion_physical_optimizer::PhysicalOptimizerRule; - use datafusion_physical_plan::limit::GlobalLimitExec; + use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use rstest::rstest; @@ -1077,7 +1077,6 @@ mod tests { async fn test_remove_unnecessary_sort6() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); let input = Arc::new( SortExec::new(vec![sort_expr("non_nullable_col", &schema)], source) .with_fetch(Some(2)), @@ -1104,6 +1103,39 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_remove_unnecessary_sort7() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = Arc::new(SortExec::new( + vec![sort_expr("non_nullable_col", &schema)], + source, + )); + let limit = Arc::new(LocalLimitExec::new(input, 2)); + let physical_plan = sort_exec( + vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ], + limit, + ); + + let expected_input = [ + "SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " LocalLimitExec: fetch=2", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "LocalLimitExec: fetch=2", + " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + #[tokio::test] async fn test_do_not_pushdown_through_limit() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index b8ca5fc6979a..88334db1030f 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::fmt::{Debug, Formatter}; use std::sync::Arc; use super::utils::add_sort_above; @@ -32,11 +34,13 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, JoinSide, Result}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; #[derive(Default, Clone)] pub struct ParentRequirements { @@ -63,47 +67,77 @@ pub fn assign_initial_requirements(node: &mut SortPushDown) { } } -/// Get fetch information for the operator. -fn get_fetch(plan: &Arc) -> Option { - if let Some(sort) = plan.as_any().downcast_ref::() { - sort.fetch() - } else { - None - } -} - /// Prunes unnecessary operators from the plan. pub(crate) fn prune_unnecessary_operators( plan: Arc, ) -> Result> { Ok(plan .transform_up(|p| { - if let Some(sort) = p.as_any().downcast_ref::() { - let out_ordering = sort.expr(); - if sort - .input() - .equivalence_properties() - .ordering_satisfy(out_ordering) - { - // Ordering is satisfied - let out_fetch = sort.fetch(); - let in_fetch = get_fetch(sort.input()); - let can_remove = match (out_fetch, in_fetch) { - (Some(out_fetch), Some(in_fetch)) => in_fetch >= out_fetch, - (None, _) => true, - // Cannot remove, as sort decreases number of rows after processing. - (Some(_), None) => false, - }; - if can_remove { - return Ok(Transformed::yes(sort.input().clone())); - } - } + if let Some(p) = p.as_any().downcast_ref::() { + return Ok(Transformed::yes(p.child.clone())); } Ok(Transformed::no(p)) })? .data) } +/// A dummy operator to make sure each node is visited, during top-down pass +/// even some of the nodes are removed from the plan. When an operator is removed from the plan, +/// this operator is inserted on top of it. With this insertion at the next iteration, its child will be visited +/// enabling us to not miss any node. +/// This operator should be removed after rule ends from the plan. +struct PlaceHolderOperatorExec { + child: Arc, +} + +impl Debug for PlaceHolderOperatorExec { + fn fmt(&self, _f: &mut Formatter<'_>) -> std::fmt::Result { + unreachable!() + } +} + +impl DisplayAs for PlaceHolderOperatorExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + write!(f, "PlaceHolderExec") + } +} + +impl ExecutionPlan for PlaceHolderOperatorExec { + fn name(&self) -> &str { + unreachable!() + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + self.child.properties() + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> Result> { + debug_assert_eq!(children.len(), 1); + Ok(Arc::new(Self { + child: children.swap_remove(0), + })) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + unreachable!() + } +} + pub(crate) fn pushdown_sorts( mut requirements: SortPushDown, ) -> Result> { @@ -116,16 +150,15 @@ pub(crate) fn pushdown_sorts( let satisfy_parent = plan .equivalence_properties() .ordering_satisfy_requirement(parent_reqs); - - if let Some(sort_exec) = plan.as_any().downcast_ref::() { + if plan.as_any().downcast_ref::().is_some() { let required_ordering = plan .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs) .unwrap_or_default(); if !satisfy_parent { // Make sure this `SortExec` satisfies parent requirements: - let fetch = sort_exec.fetch(); let sort_reqs = requirements.data.ordering_requirement.unwrap_or_default(); + let fetch = requirements.data.fetch; requirements = requirements.children.swap_remove(0); requirements = add_sort_above(requirements, sort_reqs, fetch); }; @@ -135,7 +168,7 @@ pub(crate) fn pushdown_sorts( if let Some(adjusted) = pushdown_requirement_to_children(&child.plan, &required_ordering)? { - let fetch = get_fetch(&child.plan); + let fetch = child.plan.fetch(); for (grand_child, order) in child.children.iter_mut().zip(adjusted) { grand_child.data = ParentRequirements { ordering_requirement: order, @@ -144,10 +177,22 @@ pub(crate) fn pushdown_sorts( } // Can push down requirements child.data = ParentRequirements { - ordering_requirement: None, + ordering_requirement: Some(required_ordering.clone()), fetch, }; - return Ok(Transformed::yes(child)); + + let new_plan = Arc::new(PlaceHolderOperatorExec { + child: child.plan.clone(), + }) as Arc; + let new_reqs = PlanContext::new( + new_plan, + ParentRequirements { + ordering_requirement: Some(required_ordering), + fetch, + }, + vec![child], + ); + return Ok(Transformed::yes(new_reqs)); } else { // Can not push down requirements requirements.children = vec![child]; @@ -198,9 +243,6 @@ fn pushdown_requirement_to_children( RequirementsCompatibility::NonCompatible => Ok(None), } } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - println!("trying to pushdown sort exec"); - // println!("parent_required: {:?}", parent_required); - // println!("sort_exec.properties().eq_properties: {:?}", sort_exec.properties().eq_properties); let sort_req = PhysicalSortRequirement::from_sort_exprs( sort_exec.properties().output_ordering().unwrap_or(&[]), ); @@ -209,12 +251,26 @@ fn pushdown_requirement_to_children( .eq_properties .requirements_compatible(parent_required, &sort_req) { - println!("compatible, parent_required: {:?}", parent_required); debug_assert!(!parent_required.is_empty()); Ok(Some(vec![Some(parent_required.to_vec())])) } else { Ok(None) } + } else if is_limit(plan) { + let output_req = PhysicalSortRequirement::from_sort_exprs( + plan.properties().output_ordering().unwrap_or(&[]), + ); + // Push down through limit only when requirement is aligned with output ordering. + if plan + .properties() + .eq_properties + .requirements_compatible(parent_required, &output_req) + { + let req = (!parent_required.is_empty()).then(|| parent_required.to_vec()); + Ok(Some(vec![req])) + } else { + Ok(None) + } } else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec @@ -257,7 +313,6 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() // TODO: Add support for Projection push down || plan.as_any().is::() - || is_limit(plan) || plan.as_any().is::() || pushdown_would_violate_requirements(parent_required, plan.as_ref()) { diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index de42a55ad350..a758918aa52f 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -224,6 +224,10 @@ impl ExecutionPlan for CoalesceBatchesExec { cache: self.cache.clone(), })) } + + fn fetch(&self) -> Option { + self.fetch + } } /// Stream for [`CoalesceBatchesExec`]. See [`CoalesceBatchesExec`] for more details. diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 5a3fc086c1f8..a6a15e46860c 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -399,6 +399,11 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { fn with_fetch(&self, _limit: Option) -> Option> { None } + + /// Gets the fetch count for the operator, `None` means there is no fetch. + fn fetch(&self) -> Option { + None + } } /// Extension trait provides an easy API to fetch various properties of diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index f3dad6afabde..c58f4d6ab6ec 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -260,6 +260,10 @@ impl ExecutionPlan for GlobalLimitExec { }; Ok(stats) } + + fn fetch(&self) -> Option { + self.fetch + } } /// LocalLimitExec applies a limit to a single partition @@ -428,6 +432,10 @@ impl ExecutionPlan for LocalLimitExec { }; Ok(stats) } + + fn fetch(&self) -> Option { + Some(self.fetch) + } } /// A Limit stream skips `skip` rows, and then fetch up to `fetch` rows. diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index eb77d7716848..be7268d195e1 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -934,6 +934,10 @@ impl ExecutionPlan for SortExec { cache: self.cache.clone(), })) } + + fn fetch(&self) -> Option { + self.fetch + } } #[cfg(test)] From eb83917b310c3f919236baadac49514af9bc6d95 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 15:27:57 +0300 Subject: [PATCH 05/30] Implement top down recursion with delete check --- .../src/physical_optimizer/enforce_sorting.rs | 13 +-- .../src/physical_optimizer/sort_pushdown.rs | 107 ++++-------------- 2 files changed, 25 insertions(+), 95 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 56ce7b70aaaa..e228e5ae3c4e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -44,8 +44,7 @@ use crate::physical_optimizer::replace_with_order_preserving_variants::{ replace_with_order_preserving_variants, OrderPreservationContext, }; use crate::physical_optimizer::sort_pushdown::{ - assign_initial_requirements, prune_unnecessary_operators, pushdown_sorts, - SortPushDown, + assign_initial_requirements, pushdown_sorts, SortPushDown, }; use crate::physical_optimizer::utils::{ is_coalesce_partitions, is_limit, is_repartition, is_sort, is_sort_preserving_merge, @@ -190,13 +189,12 @@ impl PhysicalOptimizerRule for EnforceSorting { // missed by the bottom-up traversal: let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); assign_initial_requirements(&mut sort_pushdown); - let adjusted = sort_pushdown.transform_down(pushdown_sorts)?.data; + let adjusted = pushdown_sorts(sort_pushdown)?; let plan = adjusted .plan .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?))) .data()?; - // Prune out unnecessary operators from the plan. - prune_unnecessary_operators(plan) + Ok(plan) } fn name(&self) -> &str { @@ -740,10 +738,7 @@ mod tests { let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); assign_initial_requirements(&mut sort_pushdown); - sort_pushdown - .transform_down(pushdown_sorts) - .data() - .and_then(check_integrity)?; + check_integrity(pushdown_sorts(sort_pushdown)?)?; // TODO: End state payloads will be checked here. } diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 88334db1030f..6e62e7a6e18c 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::any::Any; -use std::fmt::{Debug, Formatter}; +use std::fmt::Debug; use std::sync::Arc; use super::utils::add_sort_above; @@ -32,15 +31,13 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::tree_node::PlanContext; use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; -use datafusion_common::tree_node::{Transformed, TreeNode}; +use datafusion_common::tree_node::{ConcreteTreeNode, Transformed}; use datafusion_common::{plan_err, JoinSide, Result}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; #[derive(Default, Clone)] pub struct ParentRequirements { @@ -67,80 +64,29 @@ pub fn assign_initial_requirements(node: &mut SortPushDown) { } } -/// Prunes unnecessary operators from the plan. -pub(crate) fn prune_unnecessary_operators( - plan: Arc, -) -> Result> { - Ok(plan - .transform_up(|p| { - if let Some(p) = p.as_any().downcast_ref::() { - return Ok(Transformed::yes(p.child.clone())); - } - Ok(Transformed::no(p)) - })? - .data) -} - -/// A dummy operator to make sure each node is visited, during top-down pass -/// even some of the nodes are removed from the plan. When an operator is removed from the plan, -/// this operator is inserted on top of it. With this insertion at the next iteration, its child will be visited -/// enabling us to not miss any node. -/// This operator should be removed after rule ends from the plan. -struct PlaceHolderOperatorExec { - child: Arc, -} - -impl Debug for PlaceHolderOperatorExec { - fn fmt(&self, _f: &mut Formatter<'_>) -> std::fmt::Result { - unreachable!() - } -} - -impl DisplayAs for PlaceHolderOperatorExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { - write!(f, "PlaceHolderExec") +pub(crate) fn call_till_not_early_exit( + sort_pushdown: SortPushDown, +) -> Result> { + let (mut result, mut early_exit) = pushdown_sorts_helper(sort_pushdown)?; + while early_exit { + (result, early_exit) = pushdown_sorts_helper(result.data)?; } + Ok(result) } -impl ExecutionPlan for PlaceHolderOperatorExec { - fn name(&self) -> &str { - unreachable!() - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - self.child.properties() - } - - fn children(&self) -> Vec<&Arc> { - vec![&self.child] - } - - fn with_new_children( - self: Arc, - mut children: Vec>, - ) -> Result> { - debug_assert_eq!(children.len(), 1); - Ok(Arc::new(Self { - child: children.swap_remove(0), - })) - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - unreachable!() - } +pub(crate) fn pushdown_sorts(sort_pushdown: SortPushDown) -> Result { + let new_node = call_till_not_early_exit(sort_pushdown)?.data; + let (new_node, children) = new_node.take_children(); + let new_children = children + .into_iter() + .map(pushdown_sorts) + .collect::>>()?; + new_node.with_new_children(new_children) } -pub(crate) fn pushdown_sorts( +pub(crate) fn pushdown_sorts_helper( mut requirements: SortPushDown, -) -> Result> { +) -> Result<(Transformed, bool)> { let plan = &requirements.plan; let parent_reqs = requirements .data @@ -181,18 +127,7 @@ pub(crate) fn pushdown_sorts( fetch, }; - let new_plan = Arc::new(PlaceHolderOperatorExec { - child: child.plan.clone(), - }) as Arc; - let new_reqs = PlanContext::new( - new_plan, - ParentRequirements { - ordering_requirement: Some(required_ordering), - fetch, - }, - vec![child], - ); - return Ok(Transformed::yes(new_reqs)); + return Ok((Transformed::yes(child), true)); } else { // Can not push down requirements requirements.children = vec![child]; @@ -222,7 +157,7 @@ pub(crate) fn pushdown_sorts( requirements = add_sort_above(requirements, sort_reqs, fetch); assign_initial_requirements(&mut requirements); } - Ok(Transformed::yes(requirements)) + Ok((Transformed::yes(requirements), false)) } fn pushdown_requirement_to_children( From 0b66b15d0f56a61efd9aa0fc6a6c3e172860658d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 15:35:04 +0300 Subject: [PATCH 06/30] Minor changes --- datafusion/core/src/dataframe/mod.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index c4c5a4aa0834..7da4bf1f20d4 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -3019,11 +3019,11 @@ mod tests { assert_batches_sorted_eq!( [ - "+-----+-----+----+-------+", - "| one | two | c3 | total |", - "+-----+-----+----+-------+", - "| a | 3 | 13 | 16 |", - "+-----+-----+----+-------+" + "+-----+-----+-----+-------+", + "| one | two | c3 | total |", + "+-----+-----+-----+-------+", + "| a | 3 | -72 | -69 |", + "+-----+-----+-----+-------+", ], &df_sum_renamed ); From c769f9f4fe19120b36838ac7485a4163d6e1a8f5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 16:42:35 +0300 Subject: [PATCH 07/30] Minor changes --- .../src/physical_optimizer/sort_pushdown.rs | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 6e62e7a6e18c..031ab749bc61 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -31,7 +31,7 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::tree_node::PlanContext; use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; -use datafusion_common::tree_node::{ConcreteTreeNode, Transformed}; +use datafusion_common::tree_node::{ConcreteTreeNode, Transformed, TreeNodeRecursion}; use datafusion_common::{plan_err, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; @@ -67,9 +67,9 @@ pub fn assign_initial_requirements(node: &mut SortPushDown) { pub(crate) fn call_till_not_early_exit( sort_pushdown: SortPushDown, ) -> Result> { - let (mut result, mut early_exit) = pushdown_sorts_helper(sort_pushdown)?; - while early_exit { - (result, early_exit) = pushdown_sorts_helper(result.data)?; + let mut result = pushdown_sorts_helper(sort_pushdown)?; + while result.tnr == TreeNodeRecursion::Stop { + result = pushdown_sorts_helper(result.data)?; } Ok(result) } @@ -86,7 +86,7 @@ pub(crate) fn pushdown_sorts(sort_pushdown: SortPushDown) -> Result Result<(Transformed, bool)> { +) -> Result> { let plan = &requirements.plan; let parent_reqs = requirements .data @@ -127,7 +127,11 @@ pub(crate) fn pushdown_sorts_helper( fetch, }; - return Ok((Transformed::yes(child), true)); + return Ok(Transformed { + data: child, + transformed: true, + tnr: TreeNodeRecursion::Stop, + }); } else { // Can not push down requirements requirements.children = vec![child]; @@ -157,7 +161,7 @@ pub(crate) fn pushdown_sorts_helper( requirements = add_sort_above(requirements, sort_reqs, fetch); assign_initial_requirements(&mut requirements); } - Ok((Transformed::yes(requirements), false)) + Ok(Transformed::yes(requirements)) } fn pushdown_requirement_to_children( From 0ad70637f8f5f304166fe21b7908268a304c2d91 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 17:16:38 +0300 Subject: [PATCH 08/30] Address reviews --- .../src/physical_optimizer/enforce_sorting.rs | 45 +++++++++---------- .../src/physical_optimizer/sort_pushdown.rs | 23 ++++------ 2 files changed, 28 insertions(+), 40 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index e228e5ae3c4e..3b3db5c4fdcb 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -190,11 +190,10 @@ impl PhysicalOptimizerRule for EnforceSorting { let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); assign_initial_requirements(&mut sort_pushdown); let adjusted = pushdown_sorts(sort_pushdown)?; - let plan = adjusted + adjusted .plan .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?))) - .data()?; - Ok(plan) + .data() } fn name(&self) -> &str { @@ -281,7 +280,7 @@ fn parallelize_sorts( // executors don't require single partition), then we can replace // the `CoalescePartitionsExec` + `SortExec` cascade with a `SortExec` // + `SortPreservingMergeExec` cascade to parallelize sorting. - requirements = remove_corresponding_coalesce_in_sub_plan(requirements)?; + requirements = remove_bottleneck_in_subplan(requirements)?; // We also need to remove the self node since `remove_corresponding_coalesce_in_sub_plan` // deals with the children and their children and so on. requirements = requirements.children.swap_remove(0); @@ -299,7 +298,7 @@ fn parallelize_sorts( } else if is_coalesce_partitions(&requirements.plan) { // There is an unnecessary `CoalescePartitionsExec` in the plan. // This will handle the recursive `CoalescePartitionsExec` plans. - requirements = remove_corresponding_coalesce_in_sub_plan(requirements)?; + requirements = remove_bottleneck_in_subplan(requirements)?; // For the removal of self node which is also a `CoalescePartitionsExec`. requirements = requirements.children.swap_remove(0); @@ -484,8 +483,10 @@ fn adjust_window_sort_removal( Ok(window_tree) } -/// Removes the [`CoalescePartitionsExec`] from the plan in `node`. -fn remove_corresponding_coalesce_in_sub_plan( +/// Removes the [`CoalescePartitionsExec`] from the plan in `node`.After the removal of +/// the `CoalescePartitionsExec` from the plan. Some of the `RepartitionExec`s might turn into +/// redundant. Removes those `RepartitionExec`s from the plan on the way also. +fn remove_bottleneck_in_subplan( mut requirements: PlanWithCorrespondingCoalescePartitions, ) -> Result { let plan = &requirements.plan; @@ -506,32 +507,27 @@ fn remove_corresponding_coalesce_in_sub_plan( .into_iter() .map(|node| { if node.data { - remove_corresponding_coalesce_in_sub_plan(node) + remove_bottleneck_in_subplan(node) } else { Ok(node) } }) .collect::>()?; } - let mut new_req = requirements.update_plan_from_children()?; - if let Some(repartition) = new_req.plan.as_any().downcast_ref::() { - let mut can_remove = false; - if repartition - .input() - .output_partitioning() - .eq(repartition.partitioning()) - { - // Their partitioning same - can_remove = true; - } else if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() { - can_remove = - *n_out == repartition.input().output_partitioning().partition_count(); + let mut new_reqs = requirements.update_plan_from_children()?; + if let Some(repartition) = new_reqs.plan.as_any().downcast_ref::() { + let input_partitioning = repartition.input().output_partitioning(); + // We can remove this repartitioning operator if it is now a no-op: + let mut can_remove = input_partitioning.eq(repartition.partitioning()); + // We can also remove it if we ended up with an ineffective RR: + if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() { + can_remove |= *n_out == input_partitioning.partition_count(); } if can_remove { - new_req = new_req.children.swap_remove(0) + new_reqs = new_reqs.children.swap_remove(0) } } - Ok(new_req) + Ok(new_reqs) } /// Updates child to remove the unnecessary sort below it. @@ -558,10 +554,9 @@ fn remove_corresponding_sort_from_sub_plan( ) -> Result { // A `SortExec` is always at the bottom of the tree. if let Some(sort_exec) = node.plan.as_any().downcast_ref::() { + // Do not remove sorts with fetch: if sort_exec.fetch().is_none() { node = node.children.swap_remove(0); - } else { - // Do not remove the sort with fetch } } else { let mut any_connection = false; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 031ab749bc61..2c341d86632f 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::sync::Arc; -use super::utils::add_sort_above; +use super::utils::{add_sort_above, is_sort}; use crate::physical_optimizer::utils::{ is_limit, is_sort_preserving_merge, is_union, is_window, }; @@ -64,19 +64,12 @@ pub fn assign_initial_requirements(node: &mut SortPushDown) { } } -pub(crate) fn call_till_not_early_exit( - sort_pushdown: SortPushDown, -) -> Result> { - let mut result = pushdown_sorts_helper(sort_pushdown)?; - while result.tnr == TreeNodeRecursion::Stop { - result = pushdown_sorts_helper(result.data)?; - } - Ok(result) -} - pub(crate) fn pushdown_sorts(sort_pushdown: SortPushDown) -> Result { - let new_node = call_till_not_early_exit(sort_pushdown)?.data; - let (new_node, children) = new_node.take_children(); + let mut new_node = pushdown_sorts_helper(sort_pushdown)?; + while new_node.tnr == TreeNodeRecursion::Stop { + new_node = pushdown_sorts_helper(new_node.data)?; + } + let (new_node, children) = new_node.data.take_children(); let new_children = children .into_iter() .map(pushdown_sorts) @@ -96,7 +89,7 @@ pub(crate) fn pushdown_sorts_helper( let satisfy_parent = plan .equivalence_properties() .ordering_satisfy_requirement(parent_reqs); - if plan.as_any().downcast_ref::().is_some() { + if is_sort(plan) { let required_ordering = plan .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs) @@ -123,7 +116,7 @@ pub(crate) fn pushdown_sorts_helper( } // Can push down requirements child.data = ParentRequirements { - ordering_requirement: Some(required_ordering.clone()), + ordering_requirement: Some(required_ordering), fetch, }; From 3661f06810c1319377ceec8ce383e769113a434b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 17:48:43 +0300 Subject: [PATCH 09/30] Update comments --- .../core/src/physical_optimizer/enforce_sorting.rs | 7 ++++--- .../core/src/physical_optimizer/sort_pushdown.rs | 14 +++++++++----- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 3b3db5c4fdcb..8cd927017c9e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -483,9 +483,10 @@ fn adjust_window_sort_removal( Ok(window_tree) } -/// Removes the [`CoalescePartitionsExec`] from the plan in `node`.After the removal of -/// the `CoalescePartitionsExec` from the plan. Some of the `RepartitionExec`s might turn into -/// redundant. Removes those `RepartitionExec`s from the plan on the way also. +/// Removes parallelization-reducing, avoidable [`CoalescePartitionsExec`]s from the plan in `node`. +/// After the removal of such `CoalescePartitionsExec`s from the plan, some of the +/// `RepartitionExec`s might become redundant. Removes those `RepartitionExec`s from the plan as +/// well. fn remove_bottleneck_in_subplan( mut requirements: PlanWithCorrespondingCoalescePartitions, ) -> Result { diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 2c341d86632f..059741fa73a9 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -19,9 +19,7 @@ use std::fmt::Debug; use std::sync::Arc; use super::utils::{add_sort_above, is_sort}; -use crate::physical_optimizer::utils::{ - is_limit, is_sort_preserving_merge, is_union, is_window, -}; +use crate::physical_optimizer::utils::{is_sort_preserving_merge, is_union, is_window}; use crate::physical_plan::filter::FilterExec; use crate::physical_plan::joins::utils::calculate_join_output_ordering; use crate::physical_plan::joins::{HashJoinExec, SortMergeJoinExec}; @@ -188,11 +186,17 @@ fn pushdown_requirement_to_children( } else { Ok(None) } - } else if is_limit(plan) { + } else if plan.fetch().is_some() + && plan.maintains_input_order().len() == 1 + && plan.maintains_input_order()[0] + && plan.supports_limit_pushdown() + { let output_req = PhysicalSortRequirement::from_sort_exprs( plan.properties().output_ordering().unwrap_or(&[]), ); - // Push down through limit only when requirement is aligned with output ordering. + // Push down through operator with fetch when: + // - requirement is aligned with output ordering + // - it preserves ordering during execution if plan .properties() .eq_properties From 60967c1f3430bc7d61993f25c2065e4271515fd2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 17:59:51 +0300 Subject: [PATCH 10/30] Minor changes --- datafusion/physical-plan/src/limit.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index c58f4d6ab6ec..3bfb72d6226b 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -264,6 +264,10 @@ impl ExecutionPlan for GlobalLimitExec { fn fetch(&self) -> Option { self.fetch } + + fn supports_limit_pushdown(&self) -> bool { + true + } } /// LocalLimitExec applies a limit to a single partition @@ -436,6 +440,10 @@ impl ExecutionPlan for LocalLimitExec { fn fetch(&self) -> Option { Some(self.fetch) } + + fn supports_limit_pushdown(&self) -> bool { + true + } } /// A Limit stream skips `skip` rows, and then fetch up to `fetch` rows. From 6b87c4caca3bf3c4d8aafdfbaa9b7c2e28dcf398 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 7 Aug 2024 18:15:59 +0300 Subject: [PATCH 11/30] Make test deterministic --- datafusion/core/src/dataframe/mod.rs | 2 +- datafusion/core/src/physical_optimizer/enforce_sorting.rs | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 7da4bf1f20d4..00216e820bec 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2997,13 +2997,13 @@ mod tests { .await? .select_columns(&["c1", "c2", "c3"])? .filter(col("c2").eq(lit(3)).and(col("c1").eq(lit("a"))))? - .limit(0, Some(1))? .sort(vec![ // make the test deterministic col("c1").sort(true, true), col("c2").sort(true, true), col("c3").sort(true, true), ])? + .limit(0, Some(1))? .with_column("sum", col("c2") + col("c3"))?; let df_sum_renamed = df diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 8cd927017c9e..ffd62fd732b1 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -173,6 +173,7 @@ impl PhysicalOptimizerRule for EnforceSorting { } else { adjusted.plan }; + let plan_with_pipeline_fixer = OrderPreservationContext::new_default(new_plan); let updated_plan = plan_with_pipeline_fixer .transform_up(|plan_with_pipeline_fixer| { @@ -190,6 +191,7 @@ impl PhysicalOptimizerRule for EnforceSorting { let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); assign_initial_requirements(&mut sort_pushdown); let adjusted = pushdown_sorts(sort_pushdown)?; + adjusted .plan .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?))) From 8dd7e0a6ca661af37ce063d164abd61d0104067d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 8 Aug 2024 11:53:13 +0300 Subject: [PATCH 12/30] Add fetch info to the statistics --- .../physical-plan/src/coalesce_batches.rs | 8 +++++++- datafusion/physical-plan/src/sorts/sort.rs | 8 +++++++- datafusion/sqllogictest/test_files/order.slt | 17 +++++++++++++++++ datafusion/sqllogictest/test_files/window.slt | 9 +++------ 4 files changed, 34 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index a758918aa52f..a67aa8b1b068 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -29,6 +29,7 @@ use arrow::record_batch::RecordBatch; use arrow_array::{Array, ArrayRef}; use futures::stream::{Stream, StreamExt}; +use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; @@ -212,7 +213,12 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn statistics(&self) -> Result { - self.input.statistics() + let mut statistics = self.input.statistics()?; + // When fetch is used output rows generated will be precise. + if let Some(fetch) = self.fetch { + statistics.num_rows = Precision::Exact(fetch); + } + Ok(statistics) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index be7268d195e1..bf5692c5caa6 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -45,6 +45,7 @@ use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; use arrow_array::{Array, RecordBatchOptions, UInt32Array}; use arrow_schema::DataType; +use datafusion_common::stats::Precision; use datafusion_common::{internal_err, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -921,7 +922,12 @@ impl ExecutionPlan for SortExec { } fn statistics(&self) -> Result { - self.input.statistics() + let mut statistics = self.input.statistics()?; + // When fetch is used output rows generated will be precise. + if let Some(fetch) = self.fetch { + statistics.num_rows = Precision::Exact(fetch); + } + Ok(statistics) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 3382d5ddabda..c32061492c94 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1148,3 +1148,20 @@ SELECT (SELECT c from ordered_table ORDER BY c LIMIT 1) UNION ALL (SELECT 23 as ---- 0 23 + +# Do not increase partition number after fetch 1. As this will be unnecessary. +query TT +EXPLAIN SELECT a + b as sum1 FROM (SELECT a, b + FROM ordered_table + ORDER BY a ASC LIMIT 1 +); +---- +logical_plan +01)Projection: ordered_table.a + ordered_table.b AS sum1 +02)--Limit: skip=0, fetch=1 +03)----Sort: ordered_table.a ASC NULLS LAST, fetch=1 +04)------TableScan: ordered_table projection=[a, b] +physical_plan +01)ProjectionExec: expr=[a@0 + b@1 as sum1] +02)--SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index dfc882667617..5a7c60eb84f8 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2048,12 +2048,9 @@ logical_plan 05)--------TableScan: aggregate_test_100 projection=[c13] physical_plan 01)ProjectionExec: expr=[array_agg(aggregate_test_100.c13)@0 as array_agg1] -02)--AggregateExec: mode=Final, gby=[], aggr=[array_agg(aggregate_test_100.c13)] -03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(aggregate_test_100.c13)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST], preserve_partitioning=[false] -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true +02)--AggregateExec: mode=Single, gby=[], aggr=[array_agg(aggregate_test_100.c13)] +03)----SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST], preserve_partitioning=[false] +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true query ? From 15423ae4f2e9ab34c3e84ce653439689ff04dfa5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 8 Aug 2024 14:37:46 +0300 Subject: [PATCH 13/30] Enforce distribution use inexact count estimate also. --- .../enforce_distribution.rs | 9 +- .../physical-plan/src/coalesce_batches.rs | 15 +- datafusion/physical-plan/src/limit.rs | 221 ++++++++---------- datafusion/physical-plan/src/sorts/sort.rs | 15 +- .../sqllogictest/test_files/aggregate.slt | 7 +- .../test_files/aggregates_topk.slt | 35 ++- .../test_files/count_star_rule.slt | 14 +- .../sqllogictest/test_files/group_by.slt | 30 ++- datafusion/sqllogictest/test_files/limit.slt | 15 +- datafusion/sqllogictest/test_files/order.slt | 24 +- datafusion/sqllogictest/test_files/union.slt | 15 +- datafusion/sqllogictest/test_files/window.slt | 18 +- 12 files changed, 189 insertions(+), 229 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 1f076e448e60..598dd91e0187 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1098,11 +1098,10 @@ fn ensure_distribution( |(mut child, requirement, required_input_ordering, would_benefit, maintains)| { // Don't need to apply when the returned row count is not greater than batch size let num_rows = child.plan.statistics()?.num_rows; - let repartition_beneficial_stats = if num_rows.is_exact().unwrap_or(false) { - num_rows - .get_value() - .map(|value| value > &batch_size) - .unwrap() // safe to unwrap since is_exact() is true + let repartition_beneficial_stats = if let Some(n_rows) = num_rows.get_value() + { + // Row count estimate is larger than the batch size. + *n_rows > batch_size } else { true }; diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index a67aa8b1b068..c3406816b9cb 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -29,10 +29,10 @@ use arrow::record_batch::RecordBatch; use arrow_array::{Array, ArrayRef}; use futures::stream::{Stream, StreamExt}; -use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; +use crate::limit::statistics_with_fetch; use crate::{ DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; @@ -213,12 +213,13 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn statistics(&self) -> Result { - let mut statistics = self.input.statistics()?; - // When fetch is used output rows generated will be precise. - if let Some(fetch) = self.fetch { - statistics.num_rows = Precision::Exact(fetch); - } - Ok(statistics) + statistics_with_fetch( + self.input.statistics()?, + self.schema(), + self.fetch, + 0, + self.properties().partitioning.partition_count(), + ) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 3bfb72d6226b..246cfff40e80 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -185,80 +185,13 @@ impl ExecutionPlan for GlobalLimitExec { } fn statistics(&self) -> Result { - let input_stats = self.input.statistics()?; - let skip = self.skip; - let col_stats = Statistics::unknown_column(&self.schema()); - let fetch = self.fetch.unwrap_or(usize::MAX); - - let mut fetched_row_number_stats = Statistics { - num_rows: Precision::Exact(fetch), - column_statistics: col_stats.clone(), - total_byte_size: Precision::Absent, - }; - - let stats = match input_stats { - Statistics { - num_rows: Precision::Exact(nr), - .. - } - | Statistics { - num_rows: Precision::Inexact(nr), - .. - } => { - if nr <= skip { - // if all input data will be skipped, return 0 - let mut skip_all_rows_stats = Statistics { - num_rows: Precision::Exact(0), - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }; - if !input_stats.num_rows.is_exact().unwrap_or(false) { - // The input stats are inexact, so the output stats must be too. - skip_all_rows_stats = skip_all_rows_stats.into_inexact(); - } - skip_all_rows_stats - } else if nr <= fetch && self.skip == 0 { - // if the input does not reach the "fetch" globally, and "skip" is zero - // (meaning the input and output are identical), return input stats. - // Can input_stats still be used, but adjusted, in the "skip != 0" case? - input_stats - } else if nr - skip <= fetch { - // after "skip" input rows are skipped, the remaining rows are less than or equal to the - // "fetch" values, so `num_rows` must equal the remaining rows - let remaining_rows: usize = nr - skip; - let mut skip_some_rows_stats = Statistics { - num_rows: Precision::Exact(remaining_rows), - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }; - if !input_stats.num_rows.is_exact().unwrap_or(false) { - // The input stats are inexact, so the output stats must be too. - skip_some_rows_stats = skip_some_rows_stats.into_inexact(); - } - skip_some_rows_stats - } else { - // if the input is greater than "fetch+skip", the num_rows will be the "fetch", - // but we won't be able to predict the other statistics - if !input_stats.num_rows.is_exact().unwrap_or(false) - || self.fetch.is_none() - { - // If the input stats are inexact, the output stats must be too. - // If the fetch value is `usize::MAX` because no LIMIT was specified, - // we also can't represent it as an exact value. - fetched_row_number_stats = - fetched_row_number_stats.into_inexact(); - } - fetched_row_number_stats - } - } - _ => { - // The result output `num_rows` will always be no greater than the limit number. - // Should `num_rows` be marked as `Absent` here when the `fetch` value is large, - // as the actual `num_rows` may be far away from the `fetch` value? - fetched_row_number_stats.into_inexact() - } - }; - Ok(stats) + statistics_with_fetch( + self.input.statistics()?, + self.schema(), + self.fetch, + self.skip, + self.properties().partitioning.partition_count(), + ) } fn fetch(&self) -> Option { @@ -388,53 +321,13 @@ impl ExecutionPlan for LocalLimitExec { } fn statistics(&self) -> Result { - let input_stats = self.input.statistics()?; - let col_stats = Statistics::unknown_column(&self.schema()); - let stats = match input_stats { - // if the input does not reach the limit globally, return input stats - Statistics { - num_rows: Precision::Exact(nr), - .. - } - | Statistics { - num_rows: Precision::Inexact(nr), - .. - } if nr <= self.fetch => input_stats, - // if the input is greater than the limit, the num_row will be greater - // than the limit because the partitions will be limited separately - // the statistic - Statistics { - num_rows: Precision::Exact(nr), - .. - } if nr > self.fetch => Statistics { - num_rows: Precision::Exact(self.fetch), - // this is not actually exact, but will be when GlobalLimit is applied - // TODO stats: find a more explicit way to vehiculate this information - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }, - Statistics { - num_rows: Precision::Inexact(nr), - .. - } if nr > self.fetch => Statistics { - num_rows: Precision::Inexact(self.fetch), - // this is not actually exact, but will be when GlobalLimit is applied - // TODO stats: find a more explicit way to vehiculate this information - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }, - _ => Statistics { - // the result output row number will always be no greater than the limit number - num_rows: Precision::Inexact( - self.fetch - * self.properties().output_partitioning().partition_count(), - ), - - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }, - }; - Ok(stats) + statistics_with_fetch( + self.input.statistics()?, + self.schema(), + Some(self.fetch), + 0, + self.properties().partitioning.partition_count(), + ) } fn fetch(&self) -> Option { @@ -446,6 +339,92 @@ impl ExecutionPlan for LocalLimitExec { } } +/// Calculates the statistics for the operator when fetch and skip is used in the operator +/// (Output row count can be estimated in the presence of fetch and skip information). +/// using the input statistics information. +pub fn statistics_with_fetch( + input_stats: Statistics, + schema: SchemaRef, + fetch: Option, + skip: usize, + n_partitions: usize, +) -> Result { + let col_stats = Statistics::unknown_column(&schema); + + let num_rows = if let Some(fetch) = fetch { + Precision::Exact(fetch * n_partitions) + } else { + Precision::Absent + }; + let fetch_val = fetch.unwrap_or(usize::MAX); + let mut fetched_row_number_stats = Statistics { + num_rows, + column_statistics: col_stats.clone(), + total_byte_size: Precision::Absent, + }; + + let stats = match input_stats { + Statistics { + num_rows: Precision::Exact(nr), + .. + } + | Statistics { + num_rows: Precision::Inexact(nr), + .. + } => { + if nr <= skip { + // if all input data will be skipped, return 0 + let mut skip_all_rows_stats = Statistics { + num_rows: Precision::Exact(0), + column_statistics: col_stats, + total_byte_size: Precision::Absent, + }; + if !input_stats.num_rows.is_exact().unwrap_or(false) { + // The input stats are inexact, so the output stats must be too. + skip_all_rows_stats = skip_all_rows_stats.into_inexact(); + } + skip_all_rows_stats + } else if nr <= fetch_val && skip == 0 { + // if the input does not reach the "fetch" globally, and "skip" is zero + // (meaning the input and output are identical), return input stats. + // Can input_stats still be used, but adjusted, in the "skip != 0" case? + input_stats + } else if nr - skip <= fetch_val { + // after "skip" input rows are skipped, the remaining rows are less than or equal to the + // "fetch" values, so `num_rows` must equal the remaining rows + let remaining_rows: usize = nr - skip; + let mut skip_some_rows_stats = Statistics { + num_rows: Precision::Exact(remaining_rows * n_partitions), + column_statistics: col_stats, + total_byte_size: Precision::Absent, + }; + if !input_stats.num_rows.is_exact().unwrap_or(false) { + // The input stats are inexact, so the output stats must be too. + skip_some_rows_stats = skip_some_rows_stats.into_inexact(); + } + skip_some_rows_stats + } else { + // if the input is greater than "fetch+skip", the num_rows will be the "fetch", + // but we won't be able to predict the other statistics + if !input_stats.num_rows.is_exact().unwrap_or(false) || fetch.is_none() { + // If the input stats are inexact, the output stats must be too. + // If the fetch value is `usize::MAX` because no LIMIT was specified, + // we also can't represent it as an exact value. + fetched_row_number_stats = fetched_row_number_stats.into_inexact(); + } + fetched_row_number_stats + } + } + _ => { + // The result output `num_rows` will always be no greater than the limit number. + // Should `num_rows` be marked as `Absent` here when the `fetch` value is large, + // as the actual `num_rows` may be far away from the `fetch` value? + fetched_row_number_stats.into_inexact() + } + }; + Ok(stats) +} + /// A Limit stream skips `skip` rows, and then fetch up to `fetch` rows. pub struct LimitStream { /// The remaining number of rows to skip diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index bf5692c5caa6..a0b95488df9e 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -26,6 +26,7 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::expressions::PhysicalSortExpr; +use crate::limit::statistics_with_fetch; use crate::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; @@ -45,7 +46,6 @@ use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; use arrow_array::{Array, RecordBatchOptions, UInt32Array}; use arrow_schema::DataType; -use datafusion_common::stats::Precision; use datafusion_common::{internal_err, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -922,12 +922,13 @@ impl ExecutionPlan for SortExec { } fn statistics(&self) -> Result { - let mut statistics = self.input.statistics()?; - // When fetch is used output rows generated will be precise. - if let Some(fetch) = self.fetch { - statistics.num_rows = Precision::Exact(fetch); - } - Ok(statistics) + statistics_with_fetch( + self.input.statistics()?, + self.schema(), + self.fetch, + 0, + self.properties().partitioning.partition_count(), + ) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 8a5222143356..f549c9438fc9 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -909,10 +909,9 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] -10)------------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index 8e67f501dbd7..e17ad6f95ab2 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -50,10 +50,9 @@ physical_plan 03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] 04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query TI @@ -105,10 +104,9 @@ physical_plan 03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] 04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) desc limit 4; @@ -124,10 +122,9 @@ physical_plan 03)----SortExec: TopK(fetch=4), expr=[min(traces.timestamp)@1 DESC], preserve_partitioning=[true] 04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) asc limit 4; @@ -143,10 +140,9 @@ physical_plan 03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] 04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by trace_id asc limit 4; @@ -162,10 +158,9 @@ physical_plan 03)----SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query TI select trace_id, max(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 99d358ad17f0..2776fe558b89 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -47,10 +47,9 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count()] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] -06)----------MemoryExec: partitions=1, partition_sizes=[1] +03)----RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +04)------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] +05)--------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 0; @@ -66,10 +65,9 @@ physical_plan 03)----FilterExec: count()@1 > 0 04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count()] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index bd096f61fb5d..fe51ea9e2052 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2921,10 +2921,9 @@ physical_plan 03)----ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] 04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2957,10 +2956,9 @@ physical_plan 03)----ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] 04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -4176,10 +4174,9 @@ physical_plan 01)ProjectionExec: expr=[sum(DISTINCT t1.x)@1 as sum(DISTINCT t1.x), max(DISTINCT t1.x)@2 as max(DISTINCT t1.x)] 02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 -05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] -07)------------MemoryExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] +06)----------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; @@ -4198,11 +4195,10 @@ physical_plan 05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] 06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] 07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] -11)--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] -12)----------------------MemoryExec: partitions=1, partition_sizes=[1] +08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=1 +09)----------------AggregateExec: mode=Partial, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] +10)------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] +11)--------------------MemoryExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index dc3d444854c4..1a9098af46d3 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -366,15 +366,12 @@ logical_plan 04)------Filter: t1.a > Int32(3) 05)--------TableScan: t1 projection=[a] physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[count(*)] -02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------ProjectionExec: expr=[] -06)----------GlobalLimitExec: skip=6, fetch=3 -07)------------CoalesceBatchesExec: target_batch_size=8192, fetch=9 -08)--------------FilterExec: a@0 > 3 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=Single, gby=[], aggr=[count(*)] +02)--ProjectionExec: expr=[] +03)----GlobalLimitExec: skip=6, fetch=3 +04)------CoalesceBatchesExec: target_batch_size=8192, fetch=9 +05)--------FilterExec: a@0 > 3 +06)----------MemoryExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index c32061492c94..4f9b70ef567e 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -779,19 +779,17 @@ physical_plan 04)------ProjectionExec: expr=[0 as m, t@0 as t] 05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] -10)------------------ProjectionExec: expr=[column1@0 as t] -11)--------------------ValuesExec -12)------ProjectionExec: expr=[1 as m, t@0 as t] -13)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] -14)----------CoalesceBatchesExec: target_batch_size=8192 -15)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 -16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] -18)------------------ProjectionExec: expr=[column1@0 as t] -19)--------------------ValuesExec +07)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +09)----------------ProjectionExec: expr=[column1@0 as t] +10)------------------ValuesExec +11)------ProjectionExec: expr=[1 as m, t@0 as t] +12)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +13)----------CoalesceBatchesExec: target_batch_size=8192 +14)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 +15)--------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +16)----------------ProjectionExec: expr=[column1@0 as t] +17)------------------ValuesExec ##### # Multi column sorting with lists diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index aedbee35400c..eb8136747a9f 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -220,14 +220,13 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 -05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -06)----------UnionExec -07)------------MemoryExec: partitions=1, partition_sizes=[1] -08)------------MemoryExec: partitions=1, partition_sizes=[1] -09)------------ProjectionExec: expr=[name@0 || _new as name] -10)--------------MemoryExec: partitions=1, partition_sizes=[1] +03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=3 +04)------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +05)--------UnionExec +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----------MemoryExec: partitions=1, partition_sizes=[1] +08)----------ProjectionExec: expr=[name@0 || _new as name] +09)------------MemoryExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 5a7c60eb84f8..6e05776a69fb 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -4127,11 +4127,10 @@ physical_plan 01)ProjectionExec: expr=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------CoalesceBatchesExec: target_batch_size=4096 -07)------------FilterExec: a@0 = 1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +05)--------CoalesceBatchesExec: target_batch_size=4096 +06)----------FilterExec: a@0 = 1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4150,11 +4149,10 @@ physical_plan 01)ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] 02)--BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------CoalesceBatchesExec: target_batch_size=4096 -07)------------FilterExec: a@0 = 1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 +05)--------CoalesceBatchesExec: target_batch_size=4096 +06)----------FilterExec: a@0 = 1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT From 94fb83d758a7ff85586f449b3e03f2c046c6613e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 8 Aug 2024 15:02:44 +0300 Subject: [PATCH 14/30] Minor changes --- .../physical_optimizer/enforce_distribution.rs | 2 ++ .../src/physical_optimizer/enforce_sorting.rs | 3 +-- .../core/src/physical_optimizer/sort_pushdown.rs | 15 +++++++-------- datafusion/physical-plan/src/limit.rs | 2 +- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 598dd91e0187..d35be55eaa50 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1730,6 +1730,8 @@ pub(crate) mod tests { config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; config.optimizer.prefer_existing_sort = $PREFER_EXISTING_SORT; config.optimizer.prefer_existing_union = $PREFER_EXISTING_UNION; + // Use a small batch size, to trigger RoundRobin in tests + config.execution.batch_size = 1; // NOTE: These tests verify the joint `EnforceDistribution` + `EnforceSorting` cascade // because they were written prior to the separation of `BasicEnforcement` into diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index ffd62fd732b1..7d5bed8d27a9 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -61,8 +61,7 @@ use crate::physical_plan::{Distribution, ExecutionPlan, InputOrderMode}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::Partitioning; -use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::{Partitioning, PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; use datafusion_physical_plan::ExecutionPlanProperties; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 059741fa73a9..78283db68f9e 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -37,18 +37,18 @@ use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; +/// This is a "data class" we use within the [`EnforceSorting`] rule to push +/// down [`SortExec`] in the plan. In some cases, we can reduce the total +/// computational cost by pushing down `SortExec`s through some executors. The +/// object carries the parent required ordering, fetch value of the parent node as its data. +/// +/// [`EnforceSorting`]: crate::physical_optimizer::enforce_sorting::EnforceSorting #[derive(Default, Clone)] pub struct ParentRequirements { ordering_requirement: Option>, fetch: Option, } -/// This is a "data class" we use within the [`EnforceSorting`] rule to push -/// down [`SortExec`] in the plan. In some cases, we can reduce the total -/// computational cost by pushing down `SortExec`s through some executors. The -/// object carries the parent required ordering as its data. -/// -/// [`EnforceSorting`]: crate::physical_optimizer::enforce_sorting::EnforceSorting pub type SortPushDown = PlanContext; /// Assigns the ordering requirement of the root node to the its children. @@ -187,8 +187,7 @@ fn pushdown_requirement_to_children( Ok(None) } } else if plan.fetch().is_some() - && plan.maintains_input_order().len() == 1 - && plan.maintains_input_order()[0] + && plan.maintains_input_order().iter().all(|maintain| *maintain) && plan.supports_limit_pushdown() { let output_req = PhysicalSortRequirement::from_sort_exprs( diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 246cfff40e80..b95e62476e9a 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -816,7 +816,7 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_local_limit() -> Result<()> { let row_count = row_number_statistics_for_local_limit(4, 10).await?; - assert_eq!(row_count, Precision::Exact(10)); + assert_eq!(row_count, Precision::Exact(40)); Ok(()) } From 9053b9fde1eb46dabe4ffe7ff61b523df3f1df48 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 8 Aug 2024 15:03:15 +0300 Subject: [PATCH 15/30] Minor changes --- datafusion/core/src/physical_optimizer/sort_pushdown.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 78283db68f9e..0fd0a51f7141 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -187,7 +187,10 @@ fn pushdown_requirement_to_children( Ok(None) } } else if plan.fetch().is_some() - && plan.maintains_input_order().iter().all(|maintain| *maintain) + && plan + .maintains_input_order() + .iter() + .all(|maintain| *maintain) && plan.supports_limit_pushdown() { let output_req = PhysicalSortRequirement::from_sort_exprs( From 11715849c2ba1a17c8949014808d800d51ddf543 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 8 Aug 2024 15:14:41 +0300 Subject: [PATCH 16/30] Minor changes --- datafusion/core/src/physical_optimizer/sort_pushdown.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 0fd0a51f7141..f5bd452048f9 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -71,11 +71,11 @@ pub(crate) fn pushdown_sorts(sort_pushdown: SortPushDown) -> Result>>()?; + .collect::>()?; new_node.with_new_children(new_children) } -pub(crate) fn pushdown_sorts_helper( +fn pushdown_sorts_helper( mut requirements: SortPushDown, ) -> Result> { let plan = &requirements.plan; From 711038d31224ac0d783eff705b8c6d533401d23b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 15:37:03 +0300 Subject: [PATCH 17/30] Do not add unnecessary hash partitioning --- .../enforce_distribution.rs | 79 ++++++++++++++++--- .../sqllogictest/test_files/aggregate.slt | 11 +-- .../test_files/aggregates_topk.slt | 55 ++++--------- .../test_files/count_star_rule.slt | 20 ++--- .../sqllogictest/test_files/group_by.slt | 60 +++++--------- datafusion/sqllogictest/test_files/joins.slt | 21 ++--- datafusion/sqllogictest/test_files/limit.slt | 7 +- datafusion/sqllogictest/test_files/order.slt | 25 +++--- .../sqllogictest/test_files/predicates.slt | 5 +- .../test_files/sort_merge_join.slt | 12 +-- datafusion/sqllogictest/test_files/union.slt | 45 +++++------ datafusion/sqllogictest/test_files/window.slt | 12 +-- 12 files changed, 164 insertions(+), 188 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index d35be55eaa50..00fb132cbb00 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1031,6 +1031,50 @@ fn replace_order_preserving_variants( context.update_plan_from_children() } +fn repartition_desired_flags( + requirements: &[Distribution], + children: &[&Arc], + batch_size: usize, +) -> Result> { + debug_assert_eq!(requirements.len(), children.len()); + let mut desired_flags = vec![]; + for (child, requirement) in children.iter().zip(requirements) { + // Don't need to apply when the returned row count is not greater than batch size + let num_rows = child.statistics()?.num_rows; + let mut desired = if let Some(n_rows) = num_rows.get_value() { + // Row count estimate is larger than the batch size. + // Adding repartition is desirable for this case + *n_rows > batch_size + } else { + true + }; + let is_hash = matches!(requirement, Distribution::HashPartitioned(_)); + if is_hash && child.output_partitioning().partition_count() > 1 { + // when input partitioning is larger than 1 for hash requirement. + // re-partitioning is desired + desired = true; + } + desired_flags.push((is_hash, desired)); + } + // Align beneficial flags for hash partitions + if desired_flags + .iter() + .any(|(is_hash, beneficial)| *is_hash && *beneficial) + { + // There is at least one hash requirement, which is beneficial to add according to stats + // In this case, turn all beneficial flags for hash to true. To produce aligned children. + for (is_hash, beneficial) in &mut desired_flags { + if *is_hash { + *beneficial = true; + } + } + } + Ok(desired_flags + .into_iter() + .map(|(_is_hash, beneficial)| beneficial) + .collect()) +} + /// This function checks whether we need to add additional data exchange /// operators to satisfy distribution requirements. Since this function /// takes care of such requirements, we should avoid manually adding data @@ -1082,6 +1126,11 @@ fn ensure_distribution( } }; + let repartition_beneficial_stats = repartition_desired_flags( + &plan.required_input_distribution(), + &plan.children(), + batch_size, + )?; // This loop iterates over all the children to: // - Increase parallelism for every child if it is beneficial. // - Satisfy the distribution requirements of every child, if it is not @@ -1092,20 +1141,18 @@ fn ensure_distribution( plan.required_input_distribution().iter(), plan.required_input_ordering().iter(), plan.benefits_from_input_partitioning(), - plan.maintains_input_order() + plan.maintains_input_order(), + repartition_beneficial_stats.into_iter() ) .map( - |(mut child, requirement, required_input_ordering, would_benefit, maintains)| { - // Don't need to apply when the returned row count is not greater than batch size - let num_rows = child.plan.statistics()?.num_rows; - let repartition_beneficial_stats = if let Some(n_rows) = num_rows.get_value() - { - // Row count estimate is larger than the batch size. - *n_rows > batch_size - } else { - true - }; - + |( + mut child, + requirement, + required_input_ordering, + would_benefit, + maintains, + repartition_beneficial_stats, + )| { let add_roundrobin = enable_round_robin // Operator benefits from partitioning (e.g. filter): && (would_benefit && repartition_beneficial_stats) @@ -1133,7 +1180,13 @@ fn ensure_distribution( // to increase parallelism. child = add_roundrobin_on_top(child, target_partitions)?; } - child = add_hash_on_top(child, exprs.to_vec(), target_partitions)?; + // When repartitioning is not beneficial and input partition count is 1. Hash partitioning is unnecessary + if repartition_beneficial_stats + || child.plan.output_partitioning().partition_count() != 1 + { + child = + add_hash_on_top(child, exprs.to_vec(), target_partitions)?; + } } Distribution::UnspecifiedDistribution => { if add_roundrobin { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index f549c9438fc9..01153737c0cf 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -904,14 +904,9 @@ logical_plan 04)------TableScan: t projection=[c] physical_plan 01)ProjectionExec: expr=[median(alias1)@0 as median(DISTINCT t.c)] -02)--AggregateExec: mode=Final, gby=[], aggr=[median(alias1)] -03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] -05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +02)--AggregateExec: mode=Single, gby=[], aggr=[median(alias1)] +03)----AggregateExec: mode=SinglePartitioned, gby=[c@0 as alias1], aggr=[] +04)------MemoryExec: partitions=1, partition_sizes=[1] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index e17ad6f95ab2..ab937e0b598f 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -45,14 +45,9 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [max(traces.timestamp)@1 DESC], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[false] +02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TI @@ -99,14 +94,9 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [max(traces.timestamp)@1 DESC], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[false] +02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) desc limit 4; @@ -117,14 +107,9 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[min(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [min(traces.timestamp)@1 DESC], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[min(traces.timestamp)@1 DESC], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: TopK(fetch=4), expr=[min(traces.timestamp)@1 DESC], preserve_partitioning=[false] +02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) asc limit 4; @@ -135,14 +120,9 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [max(traces.timestamp)@1 ASC NULLS LAST], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[false] +02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by trace_id asc limit 4; @@ -153,14 +133,9 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)GlobalLimitExec: skip=0, fetch=4 -02)--SortPreservingMergeExec: [trace_id@0 ASC NULLS LAST], fetch=4 -03)----SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TI select trace_id, max(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 2776fe558b89..096a2db1c42b 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -45,11 +45,8 @@ logical_plan 01)Aggregate: groupBy=[[t1.a]], aggr=[[count(Int64(1)) AS count()]] 02)--TableScan: t1 projection=[a] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count()] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -04)------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] -05)--------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[count()] +02)--MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 0; @@ -63,11 +60,8 @@ physical_plan 01)ProjectionExec: expr=[a@0 as a, count()@1 as cnt] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: count()@1 > 0 -04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count()] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +04)------AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[count()] +05)--------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; @@ -84,10 +78,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@0 as a, count() PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as count_a] 02)--WindowAggExec: wdw=[count() PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "count() PARTITION BY [t1.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] +04)------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT a, COUNT() OVER (PARTITION BY a) AS count_a FROM t1 ORDER BY a; diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index fe51ea9e2052..20863d5114d2 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2020,15 +2020,12 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[last_value(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] -08)--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -13)------------------------MemoryExec: partitions=1, partition_sizes=[3] -14)--------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -16)------------------------MemoryExec: partitions=1, partition_sizes=[3] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] +12)----------------------MemoryExec: partitions=1, partition_sizes=[3] +13)----------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2916,14 +2913,10 @@ logical_plan 03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] -02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] +03)----AggregateExec: mode=SinglePartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +04)------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2951,14 +2944,10 @@ logical_plan 03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] -02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] +03)----AggregateExec: mode=SinglePartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +04)------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -4172,11 +4161,8 @@ logical_plan 03)----TableScan: t1 projection=[x, y] physical_plan 01)ProjectionExec: expr=[sum(DISTINCT t1.x)@1 as sum(DISTINCT t1.x), max(DISTINCT t1.x)@2 as max(DISTINCT t1.x)] -02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=1 -05)--------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] -06)----------MemoryExec: partitions=1, partition_sizes=[1] +02)--AggregateExec: mode=SinglePartitioned, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; @@ -4189,16 +4175,10 @@ logical_plan 05)--------TableScan: t1 projection=[x, y] physical_plan 01)ProjectionExec: expr=[sum(alias1)@1 as sum(DISTINCT t1.x), max(alias1)@2 as max(DISTINCT t1.x)] -02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 -05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] -06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] -07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=1 -09)----------------AggregateExec: mode=Partial, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] -10)------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] -11)--------------------MemoryExec: partitions=1, partition_sizes=[1] +02)--AggregateExec: mode=SinglePartitioned, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] +03)----AggregateExec: mode=SinglePartitioned, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] +04)------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] +05)--------MemoryExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 441ccb7d99d5..212c6269466b 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3667,16 +3667,17 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=Partitioned, join_type=Full, on=[(e@0, c@0)] 04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=Hash([e@0], 2), input_partitions=1 -06)----------ProjectionExec: expr=[1 as e, 3 as f] -07)------------PlaceholderRowExec -08)------CoalesceBatchesExec: target_batch_size=2 -09)--------RepartitionExec: partitioning=Hash([c@0], 2), input_partitions=2 -10)----------UnionExec -11)------------ProjectionExec: expr=[1 as c, 2 as d] -12)--------------PlaceholderRowExec -13)------------ProjectionExec: expr=[1 as c, 3 as d] -14)--------------PlaceholderRowExec +05)--------RepartitionExec: partitioning=Hash([e@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------ProjectionExec: expr=[1 as e, 3 as f] +08)--------------PlaceholderRowExec +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([c@0], 2), input_partitions=2 +11)----------UnionExec +12)------------ProjectionExec: expr=[1 as c, 2 as d] +13)--------------PlaceholderRowExec +14)------------ProjectionExec: expr=[1 as c, 3 as d] +15)--------------PlaceholderRowExec query IIII rowsort SELECT * FROM ( diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 1a9098af46d3..b1adaf24c5ac 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -396,11 +396,8 @@ logical_plan 01)Aggregate: groupBy=[[t1000.i]], aggr=[[]] 02)--TableScan: t1000 projection=[i] physical_plan -01)AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 -04)------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] -05)--------MemoryExec: partitions=4 +01)AggregateExec: mode=SinglePartitioned, gby=[i@0 as i], aggr=[] +02)--MemoryExec: partitions=1 statement ok set datafusion.explain.show_sizes = true; diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 4f9b70ef567e..55e61d2db644 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -774,22 +774,17 @@ logical_plan 14)--------------Values: (Int64(0)), (Int64(1)) physical_plan 01)SortPreservingMergeExec: [m@0 ASC NULLS LAST,t@1 ASC NULLS LAST] -02)--SortExec: expr=[m@0 ASC NULLS LAST,t@1 ASC NULLS LAST], preserve_partitioning=[true] -03)----InterleaveExec +02)--UnionExec +03)----SortExec: expr=[t@1 ASC NULLS LAST], preserve_partitioning=[false] 04)------ProjectionExec: expr=[0 as m, t@0 as t] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] -09)----------------ProjectionExec: expr=[column1@0 as t] -10)------------------ValuesExec -11)------ProjectionExec: expr=[1 as m, t@0 as t] -12)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] -13)----------CoalesceBatchesExec: target_batch_size=8192 -14)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=1 -15)--------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] -16)----------------ProjectionExec: expr=[column1@0 as t] -17)------------------ValuesExec +05)--------AggregateExec: mode=SinglePartitioned, gby=[t@0 as t], aggr=[] +06)----------ProjectionExec: expr=[column1@0 as t] +07)------------ValuesExec +08)----SortExec: expr=[t@1 ASC NULLS LAST], preserve_partitioning=[false] +09)------ProjectionExec: expr=[1 as m, t@0 as t] +10)--------AggregateExec: mode=SinglePartitioned, gby=[t@0 as t], aggr=[] +11)----------ProjectionExec: expr=[column1@0 as t] +12)------------ValuesExec ##### # Multi column sorting with lists diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 4695e37aa560..1a8ef8ed3550 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -776,8 +776,9 @@ physical_plan 15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 16)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true 17)------CoalesceBatchesExec: target_batch_size=8192 -18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -19)----------MemoryExec: partitions=1, partition_sizes=[1] +18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +19)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +20)------------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 6e7b50973cde..ea3088e69674 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -38,14 +38,10 @@ logical_plan 03)--TableScan: t2 projection=[a, b] physical_plan 01)SortMergeJoin: join_type=Inner, on=[(a@0, a@0)], filter=CAST(b@1 AS Int64) * 50 <= CAST(b@0 AS Int64) -02)--SortExec: expr=[a@0 ASC], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -05)--------MemoryExec: partitions=1, partition_sizes=[1] -06)--SortExec: expr=[a@0 ASC], preserve_partitioning=[true] -07)----CoalesceBatchesExec: target_batch_size=8192 -08)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 -09)--------MemoryExec: partitions=1, partition_sizes=[1] +02)--SortExec: expr=[a@0 ASC], preserve_partitioning=[false] +03)----MemoryExec: partitions=1, partition_sizes=[1] +04)--SortExec: expr=[a@0 ASC], preserve_partitioning=[false] +05)----MemoryExec: partitions=1, partition_sizes=[1] # inner join with join filter query TITI rowsort diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index eb8136747a9f..476ebe7ebebe 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -220,13 +220,14 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=3 -04)------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -05)--------UnionExec -06)----------MemoryExec: partitions=1, partition_sizes=[1] -07)----------MemoryExec: partitions=1, partition_sizes=[1] -08)----------ProjectionExec: expr=[name@0 || _new as name] -09)------------MemoryExec: partitions=1, partition_sizes=[1] +03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 +05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +06)----------UnionExec +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------------MemoryExec: partitions=1, partition_sizes=[1] +09)------------ProjectionExec: expr=[name@0 || _new as name] +10)--------------MemoryExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort @@ -562,15 +563,12 @@ logical_plan physical_plan 01)UnionExec 02)--ProjectionExec: expr=[Int64(1)@0 as a] -03)----AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1)], aggr=[], ordering_mode=Sorted -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=Hash([Int64(1)@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[1 as Int64(1)], aggr=[], ordering_mode=Sorted -07)------------PlaceholderRowExec -08)--ProjectionExec: expr=[2 as a] -09)----PlaceholderRowExec -10)--ProjectionExec: expr=[3 as a] -11)----PlaceholderRowExec +03)----AggregateExec: mode=SinglePartitioned, gby=[1 as Int64(1)], aggr=[], ordering_mode=Sorted +04)------PlaceholderRowExec +05)--ProjectionExec: expr=[2 as a] +06)----PlaceholderRowExec +07)--ProjectionExec: expr=[3 as a] +08)----PlaceholderRowExec # test UNION ALL aliases correctly with aliased subquery query TT @@ -593,15 +591,12 @@ logical_plan physical_plan 01)UnionExec 02)--ProjectionExec: expr=[count(*)@1 as count, n@0 as n] -03)----AggregateExec: mode=FinalPartitioned, gby=[n@0 as n], aggr=[count(*)], ordering_mode=Sorted -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=1 -06)----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[count(*)], ordering_mode=Sorted -07)------------ProjectionExec: expr=[5 as n] -08)--------------PlaceholderRowExec -09)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] -10)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] -11)------PlaceholderRowExec +03)----AggregateExec: mode=SinglePartitioned, gby=[n@0 as n], aggr=[count(*)], ordering_mode=Sorted +04)------ProjectionExec: expr=[5 as n] +05)--------PlaceholderRowExec +06)--ProjectionExec: expr=[1 as count, max(Int64(10))@0 as n] +07)----AggregateExec: mode=Single, gby=[], aggr=[max(Int64(10))] +08)------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/11409 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 6e05776a69fb..65958ecfd48e 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -4127,10 +4127,8 @@ physical_plan 01)ProjectionExec: expr=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 -05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------FilterExec: a@0 = 1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +04)------FilterExec: a@0 = 1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4149,10 +4147,8 @@ physical_plan 01)ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] 02)--BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=1 -05)--------CoalesceBatchesExec: target_batch_size=4096 -06)----------FilterExec: a@0 = 1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] +04)------FilterExec: a@0 = 1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT From 7e598e59bf8b880c57012713428cf4cfa892fc87 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 15:58:11 +0300 Subject: [PATCH 18/30] Minor changes --- .../enforce_distribution.rs | 72 ++++++++++++------- datafusion/sqllogictest/test_files/joins.slt | 21 +++--- .../sqllogictest/test_files/predicates.slt | 5 +- datafusion/sqllogictest/test_files/union.slt | 15 ++-- 4 files changed, 65 insertions(+), 48 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 00fb132cbb00..94ede39d750d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1031,17 +1031,30 @@ fn replace_order_preserving_variants( context.update_plan_from_children() } -fn repartition_desired_flags( +/// A struct to keep track of RepartitionRequirement status for each child node. +struct RepartitionRequirementStatus { + roundrobin_beneficial: bool, + hash_necessary: bool, +} + +/// Calculates the `RepartitionRequirementStatus` for each children to generate consistent requirements. +/// As an example, for hash exec left children might produce `RepartitionRequirementStatus{roundrobin_beneficial: true, hash_necessary: true}` +/// and right children might produce: `RepartitionRequirementStatus{roundrobin_beneficial: false, hash_necessary: false}`. +/// When target partitions=4, left child might produce `Hash(vec![expr], 4)` and right child might produce `Hash(vec![expr], 4)`. However, +/// for correct operation we need consistent hashes accross children. This util turns right child status: +/// from `RepartitionRequirementStatus{roundrobin_beneficial: false, hash_necessary: false}` into +/// `RepartitionRequirementStatus{roundrobin_beneficial: false, hash_necessary: true}` to generate consistent plan. +fn get_repartition_status_flags( requirements: &[Distribution], children: &[&Arc], batch_size: usize, -) -> Result> { +) -> Result> { debug_assert_eq!(requirements.len(), children.len()); - let mut desired_flags = vec![]; + let mut repartition_status_flags = vec![]; for (child, requirement) in children.iter().zip(requirements) { // Don't need to apply when the returned row count is not greater than batch size let num_rows = child.statistics()?.num_rows; - let mut desired = if let Some(n_rows) = num_rows.get_value() { + let roundrobin_beneficial = if let Some(n_rows) = num_rows.get_value() { // Row count estimate is larger than the batch size. // Adding repartition is desirable for this case *n_rows > batch_size @@ -1049,29 +1062,35 @@ fn repartition_desired_flags( true }; let is_hash = matches!(requirement, Distribution::HashPartitioned(_)); + let mut hash_necessary = false; if is_hash && child.output_partitioning().partition_count() > 1 { // when input partitioning is larger than 1 for hash requirement. // re-partitioning is desired - desired = true; + hash_necessary = true; } - desired_flags.push((is_hash, desired)); + repartition_status_flags.push(( + is_hash, + RepartitionRequirementStatus { + roundrobin_beneficial, + hash_necessary, + }, + )); } - // Align beneficial flags for hash partitions - if desired_flags - .iter() - .any(|(is_hash, beneficial)| *is_hash && *beneficial) - { - // There is at least one hash requirement, which is beneficial to add according to stats - // In this case, turn all beneficial flags for hash to true. To produce aligned children. - for (is_hash, beneficial) in &mut desired_flags { + // Align hash necessary flags for hash partitions to generate consistent hash partitions at each children + if repartition_status_flags.iter().any(|(is_hash, status)| { + *is_hash && (status.hash_necessary || status.roundrobin_beneficial) + }) { + // There is at least one hash requirement, which is necessary or beneficial according to stats + // In this case, turn all necessary flags for hash to true, to produce aligned children. + for (is_hash, status) in &mut repartition_status_flags { if *is_hash { - *beneficial = true; + status.hash_necessary = true; } } } - Ok(desired_flags + Ok(repartition_status_flags .into_iter() - .map(|(_is_hash, beneficial)| beneficial) + .map(|(_is_hash, status)| status) .collect()) } @@ -1126,7 +1145,7 @@ fn ensure_distribution( } }; - let repartition_beneficial_stats = repartition_desired_flags( + let repartition_status_flags = get_repartition_status_flags( &plan.required_input_distribution(), &plan.children(), batch_size, @@ -1142,7 +1161,7 @@ fn ensure_distribution( plan.required_input_ordering().iter(), plan.benefits_from_input_partitioning(), plan.maintains_input_order(), - repartition_beneficial_stats.into_iter() + repartition_status_flags.into_iter() ) .map( |( @@ -1151,17 +1170,20 @@ fn ensure_distribution( required_input_ordering, would_benefit, maintains, - repartition_beneficial_stats, + RepartitionRequirementStatus { + roundrobin_beneficial, + hash_necessary, + }, )| { let add_roundrobin = enable_round_robin // Operator benefits from partitioning (e.g. filter): - && (would_benefit && repartition_beneficial_stats) + && (would_benefit && roundrobin_beneficial) // Unless partitioning increases the partition count, it is not beneficial: && child.plan.output_partitioning().partition_count() < target_partitions; // When `repartition_file_scans` is set, attempt to increase // parallelism at the source. - if repartition_file_scans && repartition_beneficial_stats { + if repartition_file_scans && roundrobin_beneficial { if let Some(new_child) = child.plan.repartitioned(target_partitions, config)? { @@ -1180,10 +1202,8 @@ fn ensure_distribution( // to increase parallelism. child = add_roundrobin_on_top(child, target_partitions)?; } - // When repartitioning is not beneficial and input partition count is 1. Hash partitioning is unnecessary - if repartition_beneficial_stats - || child.plan.output_partitioning().partition_count() != 1 - { + // When inserting hash is necessary to satisy hash requirement, insert hash repartition. + if hash_necessary { child = add_hash_on_top(child, exprs.to_vec(), target_partitions)?; } diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 212c6269466b..441ccb7d99d5 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3667,17 +3667,16 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=Partitioned, join_type=Full, on=[(e@0, c@0)] 04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=Hash([e@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------ProjectionExec: expr=[1 as e, 3 as f] -08)--------------PlaceholderRowExec -09)------CoalesceBatchesExec: target_batch_size=2 -10)--------RepartitionExec: partitioning=Hash([c@0], 2), input_partitions=2 -11)----------UnionExec -12)------------ProjectionExec: expr=[1 as c, 2 as d] -13)--------------PlaceholderRowExec -14)------------ProjectionExec: expr=[1 as c, 3 as d] -15)--------------PlaceholderRowExec +05)--------RepartitionExec: partitioning=Hash([e@0], 2), input_partitions=1 +06)----------ProjectionExec: expr=[1 as e, 3 as f] +07)------------PlaceholderRowExec +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------RepartitionExec: partitioning=Hash([c@0], 2), input_partitions=2 +10)----------UnionExec +11)------------ProjectionExec: expr=[1 as c, 2 as d] +12)--------------PlaceholderRowExec +13)------------ProjectionExec: expr=[1 as c, 3 as d] +14)--------------PlaceholderRowExec query IIII rowsort SELECT * FROM ( diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 1a8ef8ed3550..4695e37aa560 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -776,9 +776,8 @@ physical_plan 15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 16)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true 17)------CoalesceBatchesExec: target_batch_size=8192 -18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -19)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -20)------------MemoryExec: partitions=1, partition_sizes=[1] +18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +19)----------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 476ebe7ebebe..cbc79cc2fc11 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -220,14 +220,13 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 -05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -06)----------UnionExec -07)------------MemoryExec: partitions=1, partition_sizes=[1] -08)------------MemoryExec: partitions=1, partition_sizes=[1] -09)------------ProjectionExec: expr=[name@0 || _new as name] -10)--------------MemoryExec: partitions=1, partition_sizes=[1] +03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=3 +04)------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +05)--------UnionExec +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----------MemoryExec: partitions=1, partition_sizes=[1] +08)----------ProjectionExec: expr=[name@0 || _new as name] +09)------------MemoryExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort From 12ad2c2157a3bac40d1ec3d14a14debdd1f67130 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 16:13:07 +0300 Subject: [PATCH 19/30] Add config option to use inexact row number estimates during planning --- datafusion/common/src/config.rs | 5 ++ .../enforce_distribution.rs | 12 +++- .../sqllogictest/test_files/aggregate.slt | 12 +++- .../test_files/aggregates_topk.slt | 60 ++++++++++++++----- .../test_files/count_star_rule.slt | 16 +++-- .../sqllogictest/test_files/group_by.slt | 49 ++++++++++----- .../test_files/information_schema.slt | 2 + datafusion/sqllogictest/test_files/limit.slt | 23 ++++--- datafusion/sqllogictest/test_files/order.slt | 32 ++++++---- datafusion/sqllogictest/test_files/union.slt | 15 ++--- datafusion/sqllogictest/test_files/window.slt | 23 ++++--- 11 files changed, 178 insertions(+), 71 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index b5204b343f05..47d23ba93154 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -333,6 +333,11 @@ config_namespace! { /// Number of input rows partial aggregation partition should process, before /// aggregation ratio check and trying to switch to skipping aggregation mode pub skip_partial_aggregation_probe_rows_threshold: usize, default = 100_000 + + /// Should DataFusion use row number estimate at the input to decide whether increasing + /// parallelism is beneficial or not. By default, only exact row number (not estimates) + /// are used for decision. Setting this flag to `true` will more likely produce better plans. + pub use_row_number_estimate_to_optimize_partitioning: bool, default = false } } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 94ede39d750d..4408ca45d732 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1048,6 +1048,7 @@ fn get_repartition_status_flags( requirements: &[Distribution], children: &[&Arc], batch_size: usize, + should_use_estimates: bool, ) -> Result> { debug_assert_eq!(requirements.len(), children.len()); let mut repartition_status_flags = vec![]; @@ -1057,7 +1058,12 @@ fn get_repartition_status_flags( let roundrobin_beneficial = if let Some(n_rows) = num_rows.get_value() { // Row count estimate is larger than the batch size. // Adding repartition is desirable for this case - *n_rows > batch_size + // According to `should_use_estimates` flag, we can either use exact and inexact row numbers or only exact row numbers for this decision. + if should_use_estimates || num_rows.is_exact().unwrap() { + *n_rows > batch_size + } else { + true + } } else { true }; @@ -1113,6 +1119,9 @@ fn ensure_distribution( let enable_round_robin = config.optimizer.enable_round_robin_repartition; let repartition_file_scans = config.optimizer.repartition_file_scans; let batch_size = config.execution.batch_size; + let should_use_estimates = config + .execution + .use_row_number_estimate_to_optimize_partitioning; let is_unbounded = dist_context.plan.execution_mode().is_unbounded(); // Use order preserving variants either of the conditions true // - it is desired according to config @@ -1149,6 +1158,7 @@ fn ensure_distribution( &plan.required_input_distribution(), &plan.children(), batch_size, + should_use_estimates, )?; // This loop iterates over all the children to: // - Increase parallelism for every child if it is beneficial. diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 01153737c0cf..8a5222143356 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -904,9 +904,15 @@ logical_plan 04)------TableScan: t projection=[c] physical_plan 01)ProjectionExec: expr=[median(alias1)@0 as median(DISTINCT t.c)] -02)--AggregateExec: mode=Single, gby=[], aggr=[median(alias1)] -03)----AggregateExec: mode=SinglePartitioned, gby=[c@0 as alias1], aggr=[] -04)------MemoryExec: partitions=1, partition_sizes=[1] +02)--AggregateExec: mode=Final, gby=[], aggr=[median(alias1)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[median(alias1)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] +10)------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index ab937e0b598f..8e67f501dbd7 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -45,9 +45,15 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[false] -02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -03)----MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [max(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TI @@ -94,9 +100,15 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[false] -02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] -03)----MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [max(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 DESC], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MIN(timestamp) from traces group by trace_id order by MIN(timestamp) desc limit 4; @@ -107,9 +119,15 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[min(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)SortExec: TopK(fetch=4), expr=[min(traces.timestamp)@1 DESC], preserve_partitioning=[false] -02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] -03)----MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [min(traces.timestamp)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[min(traces.timestamp)@1 DESC], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by MAX(timestamp) asc limit 4; @@ -120,9 +138,15 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -03)----MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [max(traces.timestamp)@1 ASC NULLS LAST], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[max(traces.timestamp)@1 ASC NULLS LAST], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select trace_id, MAX(timestamp) from traces group by trace_id order by trace_id asc limit 4; @@ -133,9 +157,15 @@ logical_plan 03)----Aggregate: groupBy=[[traces.trace_id]], aggr=[[max(traces.timestamp)]] 04)------TableScan: traces projection=[trace_id, timestamp] physical_plan -01)SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--AggregateExec: mode=SinglePartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] -03)----MemoryExec: partitions=1, partition_sizes=[1] +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [trace_id@0 ASC NULLS LAST], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[trace_id@0 ASC NULLS LAST], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TI select trace_id, max(timestamp) from traces group by trace_id order by MAX(timestamp) desc limit 4; diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 096a2db1c42b..b552e6053769 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -45,8 +45,12 @@ logical_plan 01)Aggregate: groupBy=[[t1.a]], aggr=[[count(Int64(1)) AS count()]] 02)--TableScan: t1 projection=[a] physical_plan -01)AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[count()] -02)--MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count()] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] +06)----------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 0; @@ -60,8 +64,12 @@ physical_plan 01)ProjectionExec: expr=[a@0 as a, count()@1 as cnt] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: count()@1 > 0 -04)------AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[count()] -05)--------MemoryExec: partitions=1, partition_sizes=[1] +04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count()] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, COUNT() AS cnt FROM t1 GROUP BY t1.a HAVING COUNT() > 1; diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 20863d5114d2..a4a886c75a77 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2913,10 +2913,15 @@ logical_plan 03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -01)SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] -03)----AggregateExec: mode=SinglePartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -04)------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2944,10 +2949,15 @@ logical_plan 03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -01)SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -03)----AggregateExec: mode=SinglePartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -04)------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----ProjectionExec: expr=[country@0 as country, first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -4161,8 +4171,12 @@ logical_plan 03)----TableScan: t1 projection=[x, y] physical_plan 01)ProjectionExec: expr=[sum(DISTINCT t1.x)@1 as sum(DISTINCT t1.x), max(DISTINCT t1.x)@2 as max(DISTINCT t1.x)] -02)--AggregateExec: mode=SinglePartitioned, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] -03)----MemoryExec: partitions=1, partition_sizes=[1] +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] +07)------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; @@ -4175,10 +4189,17 @@ logical_plan 05)--------TableScan: t1 projection=[x, y] physical_plan 01)ProjectionExec: expr=[sum(alias1)@1 as sum(DISTINCT t1.x), max(alias1)@2 as max(DISTINCT t1.x)] -02)--AggregateExec: mode=SinglePartitioned, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] -03)----AggregateExec: mode=SinglePartitioned, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] -04)------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] -05)--------MemoryExec: partitions=1, partition_sizes=[1] +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[sum(alias1), max(alias1)] +06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] +11)--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] +12)----------------------MemoryExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 0cbbbf3c608c..ea010efb22ef 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -215,6 +215,7 @@ datafusion.execution.sort_spill_reservation_bytes 10485760 datafusion.execution.split_file_groups_by_statistics false datafusion.execution.target_partitions 7 datafusion.execution.time_zone +00:00 +datafusion.execution.use_row_number_estimate_to_optimize_partitioning false datafusion.explain.logical_plan_only false datafusion.explain.physical_plan_only false datafusion.explain.show_schema false @@ -304,6 +305,7 @@ datafusion.execution.sort_spill_reservation_bytes 10485760 Specifies the reserve datafusion.execution.split_file_groups_by_statistics false Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental datafusion.execution.target_partitions 7 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.use_row_number_estimate_to_optimize_partitioning false Should DataFusion use row number estimate at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row number (not estimates) are used for decision. Setting this flag to `true` will more likely produce better plans. 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.explain.show_schema false When set to true, the explain statement will print schema information diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index b1adaf24c5ac..a4864f2d7b4a 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -366,12 +366,15 @@ logical_plan 04)------Filter: t1.a > Int32(3) 05)--------TableScan: t1 projection=[a] physical_plan -01)AggregateExec: mode=Single, gby=[], aggr=[count(*)] -02)--ProjectionExec: expr=[] -03)----GlobalLimitExec: skip=6, fetch=3 -04)------CoalesceBatchesExec: target_batch_size=8192, fetch=9 -05)--------FilterExec: a@0 > 3 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=Final, gby=[], aggr=[count(*)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------ProjectionExec: expr=[] +06)----------GlobalLimitExec: skip=6, fetch=3 +07)------------CoalesceBatchesExec: target_batch_size=8192, fetch=9 +08)--------------FilterExec: a@0 > 3 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query I SELECT COUNT(*) FROM (SELECT a FROM t1 WHERE a > 3 LIMIT 3 OFFSET 6); @@ -396,8 +399,12 @@ logical_plan 01)Aggregate: groupBy=[[t1000.i]], aggr=[[]] 02)--TableScan: t1000 projection=[i] physical_plan -01)AggregateExec: mode=SinglePartitioned, gby=[i@0 as i], aggr=[] -02)--MemoryExec: partitions=1 +01)AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] +06)----------MemoryExec: partitions=1 statement ok set datafusion.explain.show_sizes = true; diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 55e61d2db644..4538416427b4 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -774,17 +774,24 @@ logical_plan 14)--------------Values: (Int64(0)), (Int64(1)) physical_plan 01)SortPreservingMergeExec: [m@0 ASC NULLS LAST,t@1 ASC NULLS LAST] -02)--UnionExec -03)----SortExec: expr=[t@1 ASC NULLS LAST], preserve_partitioning=[false] +02)--SortExec: expr=[m@0 ASC NULLS LAST,t@1 ASC NULLS LAST], preserve_partitioning=[true] +03)----InterleaveExec 04)------ProjectionExec: expr=[0 as m, t@0 as t] -05)--------AggregateExec: mode=SinglePartitioned, gby=[t@0 as t], aggr=[] -06)----------ProjectionExec: expr=[column1@0 as t] -07)------------ValuesExec -08)----SortExec: expr=[t@1 ASC NULLS LAST], preserve_partitioning=[false] -09)------ProjectionExec: expr=[1 as m, t@0 as t] -10)--------AggregateExec: mode=SinglePartitioned, gby=[t@0 as t], aggr=[] -11)----------ProjectionExec: expr=[column1@0 as t] -12)------------ValuesExec +05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +10)------------------ProjectionExec: expr=[column1@0 as t] +11)--------------------ValuesExec +12)------ProjectionExec: expr=[1 as m, t@0 as t] +13)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] +14)----------CoalesceBatchesExec: target_batch_size=8192 +15)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 +16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +17)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] +18)------------------ProjectionExec: expr=[column1@0 as t] +19)--------------------ValuesExec ##### # Multi column sorting with lists @@ -1156,5 +1163,6 @@ logical_plan 04)------TableScan: ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] -02)--SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index cbc79cc2fc11..476ebe7ebebe 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -220,13 +220,14 @@ logical_plan physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=3 -04)------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -05)--------UnionExec -06)----------MemoryExec: partitions=1, partition_sizes=[1] -07)----------MemoryExec: partitions=1, partition_sizes=[1] -08)----------ProjectionExec: expr=[name@0 || _new as name] -09)------------MemoryExec: partitions=1, partition_sizes=[1] +03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 +05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] +06)----------UnionExec +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------------MemoryExec: partitions=1, partition_sizes=[1] +09)------------ProjectionExec: expr=[name@0 || _new as name] +10)--------------MemoryExec: partitions=1, partition_sizes=[1] # nested_union_all query T rowsort diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 65958ecfd48e..dfc882667617 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2048,9 +2048,12 @@ logical_plan 05)--------TableScan: aggregate_test_100 projection=[c13] physical_plan 01)ProjectionExec: expr=[array_agg(aggregate_test_100.c13)@0 as array_agg1] -02)--AggregateExec: mode=Single, gby=[], aggr=[array_agg(aggregate_test_100.c13)] -03)----SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true +02)--AggregateExec: mode=Final, gby=[], aggr=[array_agg(aggregate_test_100.c13)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(aggregate_test_100.c13)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST], preserve_partitioning=[false] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true query ? @@ -4127,8 +4130,11 @@ physical_plan 01)ProjectionExec: expr=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------FilterExec: a@0 = 1 -05)--------MemoryExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CoalesceBatchesExec: target_batch_size=4096 +07)------------FilterExec: a@0 = 1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4147,8 +4153,11 @@ physical_plan 01)ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] 02)--BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------FilterExec: a@0 = 1 -05)--------MemoryExec: partitions=1, partition_sizes=[1] +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CoalesceBatchesExec: target_batch_size=4096 +07)------------FilterExec: a@0 = 1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT From 2e3cc5d45a514f0d2f84899dadacbddf30a74313 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 16:37:58 +0300 Subject: [PATCH 20/30] Update config --- docs/source/user-guide/configs.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index badd07822ac2..0073ac1324d5 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -91,6 +91,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | | datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | | datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | +| datafusion.execution.use_row_number_estimate_to_optimize_partitioning | false | Should DataFusion use row number estimate at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row number (not estimates) are used for decision. Setting this flag to `true` will more likely produce better plans. | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | From 34af8ba54a33d0cfcc52fa278f522b0f2e7601f9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 17:19:15 +0300 Subject: [PATCH 21/30] Minor changes --- datafusion/sqllogictest/test_files/order.slt | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 4538416427b4..ba111155e267 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1149,6 +1149,9 @@ SELECT (SELECT c from ordered_table ORDER BY c LIMIT 1) UNION ALL (SELECT 23 as 0 23 +statement ok +set datafusion.execution.use_row_number_estimate_to_optimize_partitioning = true; + # Do not increase partition number after fetch 1. As this will be unnecessary. query TT EXPLAIN SELECT a + b as sum1 FROM (SELECT a, b @@ -1163,6 +1166,8 @@ logical_plan 04)------TableScan: ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true +02)--SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true + +statement ok +set datafusion.execution.use_row_number_estimate_to_optimize_partitioning = false; From 98760bcc98452a3f0f386d74c00238aa75a7ec8e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 17:19:52 +0300 Subject: [PATCH 22/30] Minor changes --- datafusion/sqllogictest/test_files/order.slt | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index ba111155e267..8937ebca0dc5 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1171,3 +1171,21 @@ physical_plan statement ok set datafusion.execution.use_row_number_estimate_to_optimize_partitioning = false; + +# Iartition number after fetch 1. Since this estimate is not exact. +query TT +EXPLAIN SELECT a + b as sum1 FROM (SELECT a, b + FROM ordered_table + ORDER BY a ASC LIMIT 1 +); +---- +logical_plan +01)Projection: ordered_table.a + ordered_table.b AS sum1 +02)--Limit: skip=0, fetch=1 +03)----Sort: ordered_table.a ASC NULLS LAST, fetch=1 +04)------TableScan: ordered_table projection=[a, b] +physical_plan +01)ProjectionExec: expr=[a@0 + b@1 as sum1] +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true From 1e4dadabd867095d347fffc1a4d411ba032f13cf Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 9 Aug 2024 17:27:50 +0300 Subject: [PATCH 23/30] Final review --- datafusion/common/src/config.rs | 10 +- .../enforce_distribution.rs | 92 +++++++++++-------- .../src/physical_optimizer/enforce_sorting.rs | 8 +- .../src/physical_optimizer/sort_pushdown.rs | 5 +- .../test_files/information_schema.slt | 4 +- docs/source/user-guide/configs.md | 2 +- 6 files changed, 72 insertions(+), 49 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 47d23ba93154..1565433259bf 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -334,10 +334,12 @@ config_namespace! { /// aggregation ratio check and trying to switch to skipping aggregation mode pub skip_partial_aggregation_probe_rows_threshold: usize, default = 100_000 - /// Should DataFusion use row number estimate at the input to decide whether increasing - /// parallelism is beneficial or not. By default, only exact row number (not estimates) - /// are used for decision. Setting this flag to `true` will more likely produce better plans. - pub use_row_number_estimate_to_optimize_partitioning: bool, default = false + /// Should DataFusion use row number estimates at the input to decide + /// whether increasing parallelism is beneficial or not. By default, + /// only exact row numbers (not estimates) are used for decision. + /// Setting this flag to `true` will likely produce better plans. + /// We plan to make this the default in the future. + pub use_row_number_estimates_to_optimize_partitioning: bool, default = false } } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 4408ca45d732..9828551977d4 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -44,6 +44,7 @@ use crate::physical_plan::windows::WindowAggExec; use crate::physical_plan::{Distribution, ExecutionPlan, Partitioning}; use arrow::compute::SortOptions; +use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::expressions::{Column, NoOp}; @@ -1031,19 +1032,46 @@ fn replace_order_preserving_variants( context.update_plan_from_children() } -/// A struct to keep track of RepartitionRequirement status for each child node. +/// A struct to keep track of repartition requirements for each child node. struct RepartitionRequirementStatus { - roundrobin_beneficial: bool, + /// Designates whether round robin partitioning is beneficial according to + /// the statistical information we have on the number of rows. + roundrobin_beneficial_stats: bool, + /// Designates whether hash partitioning is necessary. hash_necessary: bool, } -/// Calculates the `RepartitionRequirementStatus` for each children to generate consistent requirements. -/// As an example, for hash exec left children might produce `RepartitionRequirementStatus{roundrobin_beneficial: true, hash_necessary: true}` -/// and right children might produce: `RepartitionRequirementStatus{roundrobin_beneficial: false, hash_necessary: false}`. -/// When target partitions=4, left child might produce `Hash(vec![expr], 4)` and right child might produce `Hash(vec![expr], 4)`. However, -/// for correct operation we need consistent hashes accross children. This util turns right child status: -/// from `RepartitionRequirementStatus{roundrobin_beneficial: false, hash_necessary: false}` into -/// `RepartitionRequirementStatus{roundrobin_beneficial: false, hash_necessary: true}` to generate consistent plan. +/// Calculates the `RepartitionRequirementStatus` for each children to generate +/// consistent and sensible (in terms of performance) distribution requirements. +/// As an example, a hash join's left (build) child might produce +/// +/// ```text +/// RepartitionRequirementStatus { +/// roundrobin_beneficial: true, +/// hash_necessary: true +/// } +/// ``` +/// +/// while its right (probe) child might have very few rows and produce: +/// +/// ```text +/// RepartitionRequirementStatus { +/// roundrobin_beneficial: false, +/// hash_necessary: false +/// } +/// ``` +/// +/// These statuses are not consistent as all children should agree on hash +/// partitioning. This function aligns the statuses to generate consistent +/// hash partitions for each children. After alignment, the right child's +/// status would turn into: +/// +/// ```text +/// RepartitionRequirementStatus { +/// roundrobin_beneficial: false, +/// hash_necessary: true +/// } +/// ``` fn get_repartition_status_flags( requirements: &[Distribution], children: &[&Arc], @@ -1053,41 +1081,33 @@ fn get_repartition_status_flags( debug_assert_eq!(requirements.len(), children.len()); let mut repartition_status_flags = vec![]; for (child, requirement) in children.iter().zip(requirements) { - // Don't need to apply when the returned row count is not greater than batch size - let num_rows = child.statistics()?.num_rows; - let roundrobin_beneficial = if let Some(n_rows) = num_rows.get_value() { - // Row count estimate is larger than the batch size. - // Adding repartition is desirable for this case - // According to `should_use_estimates` flag, we can either use exact and inexact row numbers or only exact row numbers for this decision. - if should_use_estimates || num_rows.is_exact().unwrap() { - *n_rows > batch_size - } else { - true - } - } else { - true + // Decide whether adding a round robin is beneficial depending on + // the statistical information we have on the number of rows: + let roundrobin_beneficial = match child.statistics()?.num_rows { + Precision::Exact(n_rows) => n_rows > batch_size, + Precision::Inexact(n_rows) => !should_use_estimates || (n_rows > batch_size), + Precision::Absent => true, }; let is_hash = matches!(requirement, Distribution::HashPartitioned(_)); - let mut hash_necessary = false; - if is_hash && child.output_partitioning().partition_count() > 1 { - // when input partitioning is larger than 1 for hash requirement. - // re-partitioning is desired - hash_necessary = true; - } + // Hash re-partitioning is necessary when the input has more than one + // partitions: + let hash_necessary = is_hash && child.output_partitioning().partition_count() > 1; repartition_status_flags.push(( is_hash, RepartitionRequirementStatus { - roundrobin_beneficial, + roundrobin_beneficial_stats: roundrobin_beneficial, hash_necessary, }, )); } - // Align hash necessary flags for hash partitions to generate consistent hash partitions at each children + // Align hash necessary flags for hash partitions to generate consistent + // hash partitions at each children: if repartition_status_flags.iter().any(|(is_hash, status)| { - *is_hash && (status.hash_necessary || status.roundrobin_beneficial) + *is_hash && (status.hash_necessary || status.roundrobin_beneficial_stats) }) { - // There is at least one hash requirement, which is necessary or beneficial according to stats - // In this case, turn all necessary flags for hash to true, to produce aligned children. + // When there is at least one hash requirement that is necessary or + // beneficial according to statistics, make all children require hash + // repartitioning: for (is_hash, status) in &mut repartition_status_flags { if *is_hash { status.hash_necessary = true; @@ -1096,7 +1116,7 @@ fn get_repartition_status_flags( } Ok(repartition_status_flags .into_iter() - .map(|(_is_hash, status)| status) + .map(|(_, status)| status) .collect()) } @@ -1121,7 +1141,7 @@ fn ensure_distribution( let batch_size = config.execution.batch_size; let should_use_estimates = config .execution - .use_row_number_estimate_to_optimize_partitioning; + .use_row_number_estimates_to_optimize_partitioning; let is_unbounded = dist_context.plan.execution_mode().is_unbounded(); // Use order preserving variants either of the conditions true // - it is desired according to config @@ -1181,7 +1201,7 @@ fn ensure_distribution( would_benefit, maintains, RepartitionRequirementStatus { - roundrobin_beneficial, + roundrobin_beneficial_stats: roundrobin_beneficial, hash_necessary, }, )| { diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 7d5bed8d27a9..bce2b1bb7820 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -484,10 +484,10 @@ fn adjust_window_sort_removal( Ok(window_tree) } -/// Removes parallelization-reducing, avoidable [`CoalescePartitionsExec`]s from the plan in `node`. -/// After the removal of such `CoalescePartitionsExec`s from the plan, some of the -/// `RepartitionExec`s might become redundant. Removes those `RepartitionExec`s from the plan as -/// well. +/// Removes parallelization-reducing, avoidable [`CoalescePartitionsExec`]s from +/// the plan in `node`. After the removal of such `CoalescePartitionsExec`s from +/// the plan, some of the remaining `RepartitionExec`s might become unnecessary. +/// Removes such `RepartitionExec`s from the plan as well. fn remove_bottleneck_in_subplan( mut requirements: PlanWithCorrespondingCoalescePartitions, ) -> Result { diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index f5bd452048f9..a575d0915b0a 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -40,7 +40,8 @@ use datafusion_physical_expr::{ /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total /// computational cost by pushing down `SortExec`s through some executors. The -/// object carries the parent required ordering, fetch value of the parent node as its data. +/// object carries the parent required ordering and the (optional) fetch value +/// of the parent node as its data. /// /// [`EnforceSorting`]: crate::physical_optimizer::enforce_sorting::EnforceSorting #[derive(Default, Clone)] @@ -187,11 +188,11 @@ fn pushdown_requirement_to_children( Ok(None) } } else if plan.fetch().is_some() + && plan.supports_limit_pushdown() && plan .maintains_input_order() .iter() .all(|maintain| *maintain) - && plan.supports_limit_pushdown() { let output_req = PhysicalSortRequirement::from_sort_exprs( plan.properties().output_ordering().unwrap_or(&[]), diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index ea010efb22ef..f0191f9b9b39 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -215,7 +215,7 @@ datafusion.execution.sort_spill_reservation_bytes 10485760 datafusion.execution.split_file_groups_by_statistics false datafusion.execution.target_partitions 7 datafusion.execution.time_zone +00:00 -datafusion.execution.use_row_number_estimate_to_optimize_partitioning false +datafusion.execution.use_row_number_estimates_to_optimize_partitioning false datafusion.explain.logical_plan_only false datafusion.explain.physical_plan_only false datafusion.explain.show_schema false @@ -305,7 +305,7 @@ datafusion.execution.sort_spill_reservation_bytes 10485760 Specifies the reserve datafusion.execution.split_file_groups_by_statistics false Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental datafusion.execution.target_partitions 7 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.use_row_number_estimate_to_optimize_partitioning false Should DataFusion use row number estimate at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row number (not estimates) are used for decision. Setting this flag to `true` will more likely produce better plans. +datafusion.execution.use_row_number_estimates_to_optimize_partitioning false Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for decision. Setting this flag to `true` will likely produce better plans. We plan to make this the default in the future. 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.explain.show_schema false When set to true, the explain statement will print schema information diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 0073ac1324d5..8f59491690bd 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -91,7 +91,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | | datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | | datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| datafusion.execution.use_row_number_estimate_to_optimize_partitioning | false | Should DataFusion use row number estimate at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row number (not estimates) are used for decision. Setting this flag to `true` will more likely produce better plans. | +| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for decision. Setting this flag to `true` will likely produce better plans. We plan to make this the default in the future. | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | From 9fc4f3d325f287a6c59cdef7dcd532a875ab8ca3 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 18:04:07 +0300 Subject: [PATCH 24/30] Address reviews --- datafusion/common/src/config.rs | 3 +- datafusion/common/src/stats.rs | 88 +++++++++++++++++- .../enforce_distribution.rs | 4 +- .../src/physical_optimizer/enforce_sorting.rs | 31 +++++++ .../src/physical_optimizer/sort_pushdown.rs | 2 +- .../physical-plan/src/coalesce_batches.rs | 3 +- datafusion/physical-plan/src/limit.rs | 91 +------------------ datafusion/physical-plan/src/sorts/sort.rs | 3 +- 8 files changed, 127 insertions(+), 98 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 1565433259bf..c48845c061e7 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -336,8 +336,9 @@ config_namespace! { /// Should DataFusion use row number estimates at the input to decide /// whether increasing parallelism is beneficial or not. By default, - /// only exact row numbers (not estimates) are used for decision. + /// only exact row numbers (not estimates) are used for this decision. /// Setting this flag to `true` will likely produce better plans. + /// if the source of statistics is accurate. /// We plan to make this the default in the future. pub use_row_number_estimates_to_optimize_partitioning: bool, default = false } diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 6cefef8d0eb5..9a0643336673 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -21,7 +21,7 @@ use std::fmt::{self, Debug, Display}; use crate::ScalarValue; -use arrow_schema::Schema; +use arrow_schema::{Schema, SchemaRef}; /// Represents a value with a degree of certainty. `Precision` is used to /// propagate information the precision of statistical values. @@ -263,6 +263,92 @@ impl Statistics { .collect::>(), } } + + /// Calculates the statistics for the operator when fetch and skip is used in the operator + /// (Output row count can be estimated in the presence of fetch and skip information). + /// using the input statistics information. + pub fn with_fetch( + input_stats: Statistics, + schema: SchemaRef, + fetch: Option, + skip: usize, + n_partitions: usize, + ) -> crate::Result { + let col_stats = Statistics::unknown_column(&schema); + + let num_rows = if let Some(fetch) = fetch { + Precision::Exact(fetch * n_partitions) + } else { + Precision::Absent + }; + let fetch_val = fetch.unwrap_or(usize::MAX); + let mut fetched_row_number_stats = Statistics { + num_rows, + column_statistics: col_stats.clone(), + total_byte_size: Precision::Absent, + }; + + let stats = match input_stats { + Statistics { + num_rows: Precision::Exact(nr), + .. + } + | Statistics { + num_rows: Precision::Inexact(nr), + .. + } => { + if nr <= skip { + // if all input data will be skipped, return 0 + let mut skip_all_rows_stats = Statistics { + num_rows: Precision::Exact(0), + column_statistics: col_stats, + total_byte_size: Precision::Absent, + }; + if !input_stats.num_rows.is_exact().unwrap_or(false) { + // The input stats are inexact, so the output stats must be too. + skip_all_rows_stats = skip_all_rows_stats.into_inexact(); + } + skip_all_rows_stats + } else if nr <= fetch_val && skip == 0 { + // if the input does not reach the "fetch" globally, and "skip" is zero + // (meaning the input and output are identical), return input stats. + // Can input_stats still be used, but adjusted, in the "skip != 0" case? + input_stats + } else if nr - skip <= fetch_val { + // after "skip" input rows are skipped, the remaining rows are less than or equal to the + // "fetch" values, so `num_rows` must equal the remaining rows + let remaining_rows: usize = nr - skip; + let mut skip_some_rows_stats = Statistics { + num_rows: Precision::Exact(remaining_rows * n_partitions), + column_statistics: col_stats, + total_byte_size: Precision::Absent, + }; + if !input_stats.num_rows.is_exact().unwrap_or(false) { + // The input stats are inexact, so the output stats must be too. + skip_some_rows_stats = skip_some_rows_stats.into_inexact(); + } + skip_some_rows_stats + } else { + // if the input is greater than "fetch+skip", the num_rows will be the "fetch", + // but we won't be able to predict the other statistics + if !input_stats.num_rows.is_exact().unwrap_or(false) || fetch.is_none() { + // If the input stats are inexact, the output stats must be too. + // If the fetch value is `usize::MAX` because no LIMIT was specified, + // we also can't represent it as an exact value. + fetched_row_number_stats = fetched_row_number_stats.into_inexact(); + } + fetched_row_number_stats + } + } + _ => { + // The result output `num_rows` will always be no greater than the limit number. + // Should `num_rows` be marked as `Absent` here when the `fetch` value is large, + // as the actual `num_rows` may be far away from the `fetch` value? + fetched_row_number_stats.into_inexact() + } + }; + Ok(stats) + } } impl Display for Statistics { diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 9828551977d4..0ecce00b1171 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1072,7 +1072,7 @@ struct RepartitionRequirementStatus { /// hash_necessary: true /// } /// ``` -fn get_repartition_status_flags( +fn get_repartition_requirement_status( requirements: &[Distribution], children: &[&Arc], batch_size: usize, @@ -1174,7 +1174,7 @@ fn ensure_distribution( } }; - let repartition_status_flags = get_repartition_status_flags( + let repartition_status_flags = get_repartition_requirement_status( &plan.required_input_distribution(), &plan.children(), batch_size, diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index bce2b1bb7820..1e71535fabf6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -1097,6 +1097,37 @@ mod tests { #[tokio::test] async fn test_remove_unnecessary_sort7() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = Arc::new( + SortExec::new(vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ], source), + ); + + let physical_plan = Arc::new( + SortExec::new(vec![ + sort_expr("non_nullable_col", &schema), + ], input).with_fetch(Some(2)), + ) as Arc; + + let expected_input = [ + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_sort8() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); let input = Arc::new(SortExec::new( diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index a575d0915b0a..17d63a06a6f8 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -40,7 +40,7 @@ use datafusion_physical_expr::{ /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total /// computational cost by pushing down `SortExec`s through some executors. The -/// object carries the parent required ordering and the (optional) fetch value +/// object carries the parent required ordering and the (optional) `fetch` value /// of the parent node as its data. /// /// [`EnforceSorting`]: crate::physical_optimizer::enforce_sorting::EnforceSorting diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index c3406816b9cb..0be3f453d642 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -32,7 +32,6 @@ use futures::stream::{Stream, StreamExt}; use datafusion_common::Result; use datafusion_execution::TaskContext; -use crate::limit::statistics_with_fetch; use crate::{ DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream, }; @@ -213,7 +212,7 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn statistics(&self) -> Result { - statistics_with_fetch( + Statistics::with_fetch( self.input.statistics()?, self.schema(), self.fetch, diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index b95e62476e9a..1e83c31687c6 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -31,7 +31,6 @@ use crate::{DisplayFormatType, Distribution, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::stats::Precision; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; @@ -185,7 +184,7 @@ impl ExecutionPlan for GlobalLimitExec { } fn statistics(&self) -> Result { - statistics_with_fetch( + Statistics::with_fetch( self.input.statistics()?, self.schema(), self.fetch, @@ -321,7 +320,7 @@ impl ExecutionPlan for LocalLimitExec { } fn statistics(&self) -> Result { - statistics_with_fetch( + Statistics::with_fetch( self.input.statistics()?, self.schema(), Some(self.fetch), @@ -339,92 +338,6 @@ impl ExecutionPlan for LocalLimitExec { } } -/// Calculates the statistics for the operator when fetch and skip is used in the operator -/// (Output row count can be estimated in the presence of fetch and skip information). -/// using the input statistics information. -pub fn statistics_with_fetch( - input_stats: Statistics, - schema: SchemaRef, - fetch: Option, - skip: usize, - n_partitions: usize, -) -> Result { - let col_stats = Statistics::unknown_column(&schema); - - let num_rows = if let Some(fetch) = fetch { - Precision::Exact(fetch * n_partitions) - } else { - Precision::Absent - }; - let fetch_val = fetch.unwrap_or(usize::MAX); - let mut fetched_row_number_stats = Statistics { - num_rows, - column_statistics: col_stats.clone(), - total_byte_size: Precision::Absent, - }; - - let stats = match input_stats { - Statistics { - num_rows: Precision::Exact(nr), - .. - } - | Statistics { - num_rows: Precision::Inexact(nr), - .. - } => { - if nr <= skip { - // if all input data will be skipped, return 0 - let mut skip_all_rows_stats = Statistics { - num_rows: Precision::Exact(0), - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }; - if !input_stats.num_rows.is_exact().unwrap_or(false) { - // The input stats are inexact, so the output stats must be too. - skip_all_rows_stats = skip_all_rows_stats.into_inexact(); - } - skip_all_rows_stats - } else if nr <= fetch_val && skip == 0 { - // if the input does not reach the "fetch" globally, and "skip" is zero - // (meaning the input and output are identical), return input stats. - // Can input_stats still be used, but adjusted, in the "skip != 0" case? - input_stats - } else if nr - skip <= fetch_val { - // after "skip" input rows are skipped, the remaining rows are less than or equal to the - // "fetch" values, so `num_rows` must equal the remaining rows - let remaining_rows: usize = nr - skip; - let mut skip_some_rows_stats = Statistics { - num_rows: Precision::Exact(remaining_rows * n_partitions), - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }; - if !input_stats.num_rows.is_exact().unwrap_or(false) { - // The input stats are inexact, so the output stats must be too. - skip_some_rows_stats = skip_some_rows_stats.into_inexact(); - } - skip_some_rows_stats - } else { - // if the input is greater than "fetch+skip", the num_rows will be the "fetch", - // but we won't be able to predict the other statistics - if !input_stats.num_rows.is_exact().unwrap_or(false) || fetch.is_none() { - // If the input stats are inexact, the output stats must be too. - // If the fetch value is `usize::MAX` because no LIMIT was specified, - // we also can't represent it as an exact value. - fetched_row_number_stats = fetched_row_number_stats.into_inexact(); - } - fetched_row_number_stats - } - } - _ => { - // The result output `num_rows` will always be no greater than the limit number. - // Should `num_rows` be marked as `Absent` here when the `fetch` value is large, - // as the actual `num_rows` may be far away from the `fetch` value? - fetched_row_number_stats.into_inexact() - } - }; - Ok(stats) -} - /// A Limit stream skips `skip` rows, and then fetch up to `fetch` rows. pub struct LimitStream { /// The remaining number of rows to skip diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a0b95488df9e..c21bdb6a18b6 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -26,7 +26,6 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::expressions::PhysicalSortExpr; -use crate::limit::statistics_with_fetch; use crate::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; @@ -922,7 +921,7 @@ impl ExecutionPlan for SortExec { } fn statistics(&self) -> Result { - statistics_with_fetch( + Statistics::with_fetch( self.input.statistics()?, self.schema(), self.fetch, From 111605862e68b676e75dc4eb909a9038ed7de347 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 18:24:12 +0300 Subject: [PATCH 25/30] Add handling for sort removal with fetch --- .../src/physical_optimizer/enforce_sorting.rs | 17 ++++++++++++++--- .../test_files/information_schema.slt | 2 +- datafusion/sqllogictest/test_files/order.slt | 4 ++-- 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 1e71535fabf6..404f4d652a6e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -65,6 +65,7 @@ use datafusion_physical_expr::{Partitioning, PhysicalSortExpr, PhysicalSortRequi use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::limit::LocalLimitExec; use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::izip; @@ -399,8 +400,12 @@ fn analyze_immediate_sort_removal( // Replace the sort with a sort-preserving merge: let expr = sort_exec.expr().to_vec(); Arc::new(SortPreservingMergeExec::new(expr, sort_input.clone())) as _ + } else if let Some(fetch) = sort_exec.fetch(){ + // Remove the sort, when its fetch is None: + node.children = node.children.swap_remove(0).children; + Arc::new(LocalLimitExec::new(sort_input.clone(), fetch)) } else { - // Remove the sort: + // Remove the sort, when its fetch is None: node.children = node.children.swap_remove(0).children; sort_input.clone() }; @@ -1117,9 +1122,15 @@ mod tests { " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; + // TODO: We can pushdown top sort into below to produce following plan: + // let expected_optimized = [ + // "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + // " MemoryExec: partitions=1, partition_sizes=[0]", + // ]; let expected_optimized = [ - "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + "LocalLimitExec: fetch=2", + " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index f0191f9b9b39..ff793a72fd8a 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -305,7 +305,7 @@ datafusion.execution.sort_spill_reservation_bytes 10485760 Specifies the reserve datafusion.execution.split_file_groups_by_statistics false Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental datafusion.execution.target_partitions 7 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.use_row_number_estimates_to_optimize_partitioning false Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for decision. Setting this flag to `true` will likely produce better plans. We plan to make this the default in the future. +datafusion.execution.use_row_number_estimates_to_optimize_partitioning false Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. 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.explain.show_schema false When set to true, the explain statement will print schema information diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 8937ebca0dc5..db709b2a178c 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1150,7 +1150,7 @@ SELECT (SELECT c from ordered_table ORDER BY c LIMIT 1) UNION ALL (SELECT 23 as 23 statement ok -set datafusion.execution.use_row_number_estimate_to_optimize_partitioning = true; +set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = true; # Do not increase partition number after fetch 1. As this will be unnecessary. query TT @@ -1170,7 +1170,7 @@ physical_plan 03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true statement ok -set datafusion.execution.use_row_number_estimate_to_optimize_partitioning = false; +set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = false; # Iartition number after fetch 1. Since this estimate is not exact. query TT From 44dc292f82c729668ed40c4f67592fd9c109df58 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 18:26:47 +0300 Subject: [PATCH 26/30] Fix linter errors --- datafusion/common/src/stats.rs | 7 +++++-- .../src/physical_optimizer/enforce_sorting.rs | 18 +++++++++--------- datafusion/physical-plan/src/limit.rs | 1 + 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 9a0643336673..1d59cf57e93b 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -331,11 +331,14 @@ impl Statistics { } else { // if the input is greater than "fetch+skip", the num_rows will be the "fetch", // but we won't be able to predict the other statistics - if !input_stats.num_rows.is_exact().unwrap_or(false) || fetch.is_none() { + if !input_stats.num_rows.is_exact().unwrap_or(false) + || fetch.is_none() + { // If the input stats are inexact, the output stats must be too. // If the fetch value is `usize::MAX` because no LIMIT was specified, // we also can't represent it as an exact value. - fetched_row_number_stats = fetched_row_number_stats.into_inexact(); + fetched_row_number_stats = + fetched_row_number_stats.into_inexact(); } fetched_row_number_stats } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 404f4d652a6e..46b47157a07b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -62,10 +62,10 @@ use crate::physical_plan::{Distribution, ExecutionPlan, InputOrderMode}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::{Partitioning, PhysicalSortExpr, PhysicalSortRequirement}; +use datafusion_physical_plan::limit::LocalLimitExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_physical_plan::limit::LocalLimitExec; use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::izip; @@ -400,7 +400,7 @@ fn analyze_immediate_sort_removal( // Replace the sort with a sort-preserving merge: let expr = sort_exec.expr().to_vec(); Arc::new(SortPreservingMergeExec::new(expr, sort_input.clone())) as _ - } else if let Some(fetch) = sort_exec.fetch(){ + } else if let Some(fetch) = sort_exec.fetch() { // Remove the sort, when its fetch is None: node.children = node.children.swap_remove(0).children; Arc::new(LocalLimitExec::new(sort_input.clone(), fetch)) @@ -1104,17 +1104,17 @@ mod tests { async fn test_remove_unnecessary_sort7() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); - let input = Arc::new( - SortExec::new(vec![ + let input = Arc::new(SortExec::new( + vec![ sort_expr("non_nullable_col", &schema), sort_expr("nullable_col", &schema), - ], source), - ); + ], + source, + )); let physical_plan = Arc::new( - SortExec::new(vec![ - sort_expr("non_nullable_col", &schema), - ], input).with_fetch(Some(2)), + SortExec::new(vec![sort_expr("non_nullable_col", &schema)], input) + .with_fetch(Some(2)), ) as Arc; let expected_input = [ diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 1e83c31687c6..fd2a2b54cf53 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -473,6 +473,7 @@ mod tests { use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use arrow_array::RecordBatchOptions; use arrow_schema::Schema; + use datafusion_common::stats::Precision; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalExpr; From c6d2de69fb84edaea78a39ce05f33ccdc0b21382 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 18:29:15 +0300 Subject: [PATCH 27/30] Minor changes --- datafusion/core/src/physical_optimizer/enforce_sorting.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 46b47157a07b..f1377ce73bf6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -1122,11 +1122,6 @@ mod tests { " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; - // TODO: We can pushdown top sort into below to produce following plan: - // let expected_optimized = [ - // "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", - // " MemoryExec: partitions=1, partition_sizes=[0]", - // ]; let expected_optimized = [ "LocalLimitExec: fetch=2", " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", From c7c85f43f7d5f190b8b042a06a9e5605e3106580 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 9 Aug 2024 18:37:16 +0300 Subject: [PATCH 28/30] Update config --- docs/source/user-guide/configs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 8f59491690bd..2c6303063642 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -91,7 +91,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | | datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | | datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for decision. Setting this flag to `true` will likely produce better plans. We plan to make this the default in the future. | +| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | From 7c8967dad7bab439aa55676043eb6b53da4e5b3e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 9 Aug 2024 22:11:44 +0300 Subject: [PATCH 29/30] Cleanup stats under fetch --- datafusion/common/src/stats.rs | 163 +++++++++--------- datafusion/core/src/datasource/statistics.rs | 2 +- .../enforce_distribution.rs | 62 ++++--- .../src/physical_optimizer/enforce_sorting.rs | 13 +- .../physical-plan/src/coalesce_batches.rs | 8 +- datafusion/physical-plan/src/filter.rs | 2 +- datafusion/physical-plan/src/limit.rs | 8 +- datafusion/physical-plan/src/sorts/sort.rs | 8 +- datafusion/sqllogictest/test_files/order.slt | 4 +- 9 files changed, 131 insertions(+), 139 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 1d59cf57e93b..d6b5310581d7 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -19,7 +19,7 @@ use std::fmt::{self, Debug, Display}; -use crate::ScalarValue; +use crate::{Result, ScalarValue}; use arrow_schema::{Schema, SchemaRef}; @@ -247,48 +247,30 @@ impl Statistics { /// If the exactness of a [`Statistics`] instance is lost, this function relaxes /// the exactness of all information by converting them [`Precision::Inexact`]. - pub fn into_inexact(self) -> Self { - Statistics { - num_rows: self.num_rows.to_inexact(), - total_byte_size: self.total_byte_size.to_inexact(), - column_statistics: self - .column_statistics - .into_iter() - .map(|cs| ColumnStatistics { - null_count: cs.null_count.to_inexact(), - max_value: cs.max_value.to_inexact(), - min_value: cs.min_value.to_inexact(), - distinct_count: cs.distinct_count.to_inexact(), - }) - .collect::>(), - } + pub fn to_inexact(mut self) -> Self { + self.num_rows = self.num_rows.to_inexact(); + self.total_byte_size = self.total_byte_size.to_inexact(); + self.column_statistics = self + .column_statistics + .into_iter() + .map(|s| s.to_inexact()) + .collect(); + self } - /// Calculates the statistics for the operator when fetch and skip is used in the operator - /// (Output row count can be estimated in the presence of fetch and skip information). - /// using the input statistics information. + /// Calculates the statistics after `fetch` and `skip` operations apply. + /// Here, `self` denotes per-partition statistics. Use the `n_partitions` + /// parameter to compute global statistics in a multi-partition setting. pub fn with_fetch( - input_stats: Statistics, + mut self, schema: SchemaRef, fetch: Option, skip: usize, n_partitions: usize, - ) -> crate::Result { - let col_stats = Statistics::unknown_column(&schema); - - let num_rows = if let Some(fetch) = fetch { - Precision::Exact(fetch * n_partitions) - } else { - Precision::Absent - }; + ) -> Result { let fetch_val = fetch.unwrap_or(usize::MAX); - let mut fetched_row_number_stats = Statistics { - num_rows, - column_statistics: col_stats.clone(), - total_byte_size: Precision::Absent, - }; - let stats = match input_stats { + self.num_rows = match self { Statistics { num_rows: Precision::Exact(nr), .. @@ -297,60 +279,64 @@ impl Statistics { num_rows: Precision::Inexact(nr), .. } => { + // Here, the inexact case gives us an upper bound on the number of rows. if nr <= skip { - // if all input data will be skipped, return 0 - let mut skip_all_rows_stats = Statistics { - num_rows: Precision::Exact(0), - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }; - if !input_stats.num_rows.is_exact().unwrap_or(false) { - // The input stats are inexact, so the output stats must be too. - skip_all_rows_stats = skip_all_rows_stats.into_inexact(); - } - skip_all_rows_stats + // All input data will be skipped: + Precision::Exact(0) } else if nr <= fetch_val && skip == 0 { - // if the input does not reach the "fetch" globally, and "skip" is zero - // (meaning the input and output are identical), return input stats. - // Can input_stats still be used, but adjusted, in the "skip != 0" case? - input_stats + // If the input does not reach the `fetch` globally, and `skip` + // is zero (meaning the input and output are identical), return + // input stats as is. + // TODO: Can input stats still be used, but adjusted, when `skip` + // is non-zero? + return Ok(self); } else if nr - skip <= fetch_val { - // after "skip" input rows are skipped, the remaining rows are less than or equal to the - // "fetch" values, so `num_rows` must equal the remaining rows - let remaining_rows: usize = nr - skip; - let mut skip_some_rows_stats = Statistics { - num_rows: Precision::Exact(remaining_rows * n_partitions), - column_statistics: col_stats, - total_byte_size: Precision::Absent, - }; - if !input_stats.num_rows.is_exact().unwrap_or(false) { - // The input stats are inexact, so the output stats must be too. - skip_some_rows_stats = skip_some_rows_stats.into_inexact(); - } - skip_some_rows_stats + // After `skip` input rows are skipped, the remaining rows are + // less than or equal to the `fetch` values, so `num_rows` must + // equal the remaining rows. + check_num_rows( + (nr - skip).checked_mul(n_partitions), + // We know that we have an estimate for the number of rows: + self.num_rows.is_exact().unwrap(), + ) } else { - // if the input is greater than "fetch+skip", the num_rows will be the "fetch", - // but we won't be able to predict the other statistics - if !input_stats.num_rows.is_exact().unwrap_or(false) - || fetch.is_none() - { - // If the input stats are inexact, the output stats must be too. - // If the fetch value is `usize::MAX` because no LIMIT was specified, - // we also can't represent it as an exact value. - fetched_row_number_stats = - fetched_row_number_stats.into_inexact(); - } - fetched_row_number_stats + // At this point we know that we were given a `fetch` value + // as the `None` case would go into the branch above. Since + // the input has more rows than `fetch + skip`, the number + // of rows will be the `fetch`, but we won't be able to + // predict the other statistics. + check_num_rows( + fetch_val.checked_mul(n_partitions), + // We know that we have an estimate for the number of rows: + self.num_rows.is_exact().unwrap(), + ) } } - _ => { - // The result output `num_rows` will always be no greater than the limit number. - // Should `num_rows` be marked as `Absent` here when the `fetch` value is large, - // as the actual `num_rows` may be far away from the `fetch` value? - fetched_row_number_stats.into_inexact() - } + Statistics { + num_rows: Precision::Absent, + .. + } => check_num_rows(fetch.and_then(|v| v.checked_mul(n_partitions)), false), }; - Ok(stats) + self.column_statistics = Statistics::unknown_column(&schema); + self.total_byte_size = Precision::Absent; + Ok(self) + } +} + +/// Creates an estimate of the number of rows in the output using the given +/// optional value and exactness flag. +fn check_num_rows(value: Option, is_exact: bool) -> Precision { + if let Some(value) = value { + if is_exact { + Precision::Exact(value) + } else { + // If the input stats are inexact, so are the output stats. + Precision::Inexact(value) + } + } else { + // If the estimate is not available (e.g. due to an overflow), we can + // not produce a reliable estimate. + Precision::Absent } } @@ -425,14 +411,25 @@ impl ColumnStatistics { } /// Returns a [`ColumnStatistics`] instance having all [`Precision::Absent`] parameters. - pub fn new_unknown() -> ColumnStatistics { - ColumnStatistics { + pub fn new_unknown() -> Self { + Self { null_count: Precision::Absent, max_value: Precision::Absent, min_value: Precision::Absent, distinct_count: Precision::Absent, } } + + /// If the exactness of a [`ColumnStatistics`] instance is lost, this + /// function relaxes the exactness of all information by converting them + /// [`Precision::Inexact`]. + pub fn to_inexact(mut self) -> Self { + self.null_count = self.null_count.to_inexact(); + self.max_value = self.max_value.to_inexact(); + self.min_value = self.min_value.to_inexact(); + self.distinct_count = self.distinct_count.to_inexact(); + self + } } #[cfg(test)] diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 8c789e461b08..563d8456ed94 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -145,7 +145,7 @@ pub async fn get_statistics_with_limit( // If we still have files in the stream, it means that the limit kicked // in, and the statistic could have been different had we processed the // files in a different order. - statistics = statistics.into_inexact() + statistics = statistics.to_inexact() } Ok((result_files, statistics)) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 0ecce00b1171..2ee5624c83dd 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1034,6 +1034,11 @@ fn replace_order_preserving_variants( /// A struct to keep track of repartition requirements for each child node. struct RepartitionRequirementStatus { + /// The distribution requirement for the node. + requirement: Distribution, + /// Designates whether round robin partitioning is theoretically beneficial; + /// i.e. the operator can actually utilize parallelism. + roundrobin_beneficial: bool, /// Designates whether round robin partitioning is beneficial according to /// the statistical information we have on the number of rows. roundrobin_beneficial_stats: bool, @@ -1047,7 +1052,7 @@ struct RepartitionRequirementStatus { /// /// ```text /// RepartitionRequirementStatus { -/// roundrobin_beneficial: true, +/// .., /// hash_necessary: true /// } /// ``` @@ -1056,7 +1061,7 @@ struct RepartitionRequirementStatus { /// /// ```text /// RepartitionRequirementStatus { -/// roundrobin_beneficial: false, +/// .., /// hash_necessary: false /// } /// ``` @@ -1068,22 +1073,26 @@ struct RepartitionRequirementStatus { /// /// ```text /// RepartitionRequirementStatus { -/// roundrobin_beneficial: false, +/// .., /// hash_necessary: true /// } /// ``` fn get_repartition_requirement_status( - requirements: &[Distribution], - children: &[&Arc], + plan: &Arc, batch_size: usize, should_use_estimates: bool, ) -> Result> { - debug_assert_eq!(requirements.len(), children.len()); + let mut needs_alignment = false; + let children = plan.children(); + let rr_beneficial = plan.benefits_from_input_partitioning(); + let requirements = plan.required_input_distribution(); let mut repartition_status_flags = vec![]; - for (child, requirement) in children.iter().zip(requirements) { + for (child, requirement, roundrobin_beneficial) in + izip!(children.into_iter(), requirements, rr_beneficial) + { // Decide whether adding a round robin is beneficial depending on // the statistical information we have on the number of rows: - let roundrobin_beneficial = match child.statistics()?.num_rows { + let roundrobin_beneficial_stats = match child.statistics()?.num_rows { Precision::Exact(n_rows) => n_rows > batch_size, Precision::Inexact(n_rows) => !should_use_estimates || (n_rows > batch_size), Precision::Absent => true, @@ -1091,20 +1100,22 @@ fn get_repartition_requirement_status( let is_hash = matches!(requirement, Distribution::HashPartitioned(_)); // Hash re-partitioning is necessary when the input has more than one // partitions: - let hash_necessary = is_hash && child.output_partitioning().partition_count() > 1; + let multi_partitions = child.output_partitioning().partition_count() > 1; + let roundrobin_sensible = roundrobin_beneficial && roundrobin_beneficial_stats; + needs_alignment |= is_hash && (multi_partitions || roundrobin_sensible); repartition_status_flags.push(( is_hash, RepartitionRequirementStatus { - roundrobin_beneficial_stats: roundrobin_beneficial, - hash_necessary, + requirement, + roundrobin_beneficial, + roundrobin_beneficial_stats, + hash_necessary: is_hash && multi_partitions, }, )); } // Align hash necessary flags for hash partitions to generate consistent // hash partitions at each children: - if repartition_status_flags.iter().any(|(is_hash, status)| { - *is_hash && (status.hash_necessary || status.roundrobin_beneficial_stats) - }) { + if needs_alignment { // When there is at least one hash requirement that is necessary or // beneficial according to statistics, make all children require hash // repartitioning: @@ -1174,12 +1185,8 @@ fn ensure_distribution( } }; - let repartition_status_flags = get_repartition_requirement_status( - &plan.required_input_distribution(), - &plan.children(), - batch_size, - should_use_estimates, - )?; + let repartition_status_flags = + get_repartition_requirement_status(&plan, batch_size, should_use_estimates)?; // This loop iterates over all the children to: // - Increase parallelism for every child if it is beneficial. // - Satisfy the distribution requirements of every child, if it is not @@ -1187,33 +1194,32 @@ fn ensure_distribution( // We store the updated children in `new_children`. let children = izip!( children.into_iter(), - plan.required_input_distribution().iter(), plan.required_input_ordering().iter(), - plan.benefits_from_input_partitioning(), plan.maintains_input_order(), repartition_status_flags.into_iter() ) .map( |( mut child, - requirement, required_input_ordering, - would_benefit, maintains, RepartitionRequirementStatus { - roundrobin_beneficial_stats: roundrobin_beneficial, + requirement, + roundrobin_beneficial, + roundrobin_beneficial_stats, hash_necessary, }, )| { let add_roundrobin = enable_round_robin // Operator benefits from partitioning (e.g. filter): - && (would_benefit && roundrobin_beneficial) + && roundrobin_beneficial + && roundrobin_beneficial_stats // Unless partitioning increases the partition count, it is not beneficial: && child.plan.output_partitioning().partition_count() < target_partitions; // When `repartition_file_scans` is set, attempt to increase // parallelism at the source. - if repartition_file_scans && roundrobin_beneficial { + if repartition_file_scans && roundrobin_beneficial_stats { if let Some(new_child) = child.plan.repartitioned(target_partitions, config)? { @@ -1222,7 +1228,7 @@ fn ensure_distribution( } // Satisfy the distribution requirement if it is unmet. - match requirement { + match &requirement { Distribution::SinglePartition => { child = add_spm_on_top(child); } diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index f1377ce73bf6..76df99b82c53 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -400,14 +400,15 @@ fn analyze_immediate_sort_removal( // Replace the sort with a sort-preserving merge: let expr = sort_exec.expr().to_vec(); Arc::new(SortPreservingMergeExec::new(expr, sort_input.clone())) as _ - } else if let Some(fetch) = sort_exec.fetch() { - // Remove the sort, when its fetch is None: - node.children = node.children.swap_remove(0).children; - Arc::new(LocalLimitExec::new(sort_input.clone(), fetch)) } else { - // Remove the sort, when its fetch is None: + // Remove the sort: node.children = node.children.swap_remove(0).children; - sort_input.clone() + if let Some(fetch) = sort_exec.fetch() { + // If the sort has a fetch, we need to add a limit: + Arc::new(LocalLimitExec::new(sort_input.clone(), fetch)) + } else { + sort_input.clone() + } }; for child in node.children.iter_mut() { child.data = false; diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 0be3f453d642..13c10c535c08 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -212,13 +212,7 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn statistics(&self) -> Result { - Statistics::with_fetch( - self.input.statistics()?, - self.schema(), - self.fetch, - 0, - self.properties().partitioning.partition_count(), - ) + Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 69bcfefcd476..fa9108057cfe 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -126,7 +126,7 @@ impl FilterExec { let schema = input.schema(); if !check_support(predicate, &schema) { let selectivity = default_selectivity as f64 / 100.0; - let mut stats = input_stats.into_inexact(); + let mut stats = input_stats.to_inexact(); stats.num_rows = stats.num_rows.with_estimated_selectivity(selectivity); stats.total_byte_size = stats .total_byte_size diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index fd2a2b54cf53..360e942226d2 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -189,7 +189,7 @@ impl ExecutionPlan for GlobalLimitExec { self.schema(), self.fetch, self.skip, - self.properties().partitioning.partition_count(), + 1, ) } @@ -325,7 +325,7 @@ impl ExecutionPlan for LocalLimitExec { self.schema(), Some(self.fetch), 0, - self.properties().partitioning.partition_count(), + 1, ) } @@ -703,7 +703,7 @@ mod tests { let row_count = row_number_inexact_statistics_for_global_limit(400, Some(10)).await?; - assert_eq!(row_count, Precision::Inexact(0)); + assert_eq!(row_count, Precision::Exact(0)); let row_count = row_number_inexact_statistics_for_global_limit(398, Some(10)).await?; @@ -730,7 +730,7 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_local_limit() -> Result<()> { let row_count = row_number_statistics_for_local_limit(4, 10).await?; - assert_eq!(row_count, Precision::Exact(40)); + assert_eq!(row_count, Precision::Exact(10)); Ok(()) } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index c21bdb6a18b6..e7e1c5481f80 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -921,13 +921,7 @@ impl ExecutionPlan for SortExec { } fn statistics(&self) -> Result { - Statistics::with_fetch( - self.input.statistics()?, - self.schema(), - self.fetch, - 0, - self.properties().partitioning.partition_count(), - ) + Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index db709b2a178c..569602166b38 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1152,7 +1152,7 @@ SELECT (SELECT c from ordered_table ORDER BY c LIMIT 1) UNION ALL (SELECT 23 as statement ok set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = true; -# Do not increase partition number after fetch 1. As this will be unnecessary. +# Do not increase the number of partitions after fetch one, as this will be unnecessary. query TT EXPLAIN SELECT a + b as sum1 FROM (SELECT a, b FROM ordered_table @@ -1172,7 +1172,7 @@ physical_plan statement ok set datafusion.execution.use_row_number_estimates_to_optimize_partitioning = false; -# Iartition number after fetch 1. Since this estimate is not exact. +# Here, we have multiple partitions after fetch one, since the row count estimate is not exact. query TT EXPLAIN SELECT a + b as sum1 FROM (SELECT a, b FROM ordered_table From ed35660a4af589d5b75225027d4d4663f9ca6d9b Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 10 Aug 2024 08:29:39 +0300 Subject: [PATCH 30/30] Update SLT comment --- datafusion/sqllogictest/test_files/limit.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index a4864f2d7b4a..4cdd40ac8c34 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -390,8 +390,8 @@ SELECT ROW_NUMBER() OVER (PARTITION BY t1.column1) FROM t t1, t t2, t t3; statement ok set datafusion.explain.show_sizes = false; -# verify that there are multiple partitions in the input (i.e. MemoryExec says -# there are 4 partitions) so that this tests multi-partition limit. +# verify that there are multiple partitions in the input so that this tests +# multi-partition limit. query TT EXPLAIN SELECT DISTINCT i FROM t1000; ----