From 096b4539607dd33dcd0620aa051ce5954b792885 Mon Sep 17 00:00:00 2001 From: blaginin Date: Thu, 16 Oct 2025 10:58:48 +0100 Subject: [PATCH 01/41] add `assert_plan` --- .../enforce_distribution.rs | 26 ++++++++++++++----- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 63111f43806b3..ba041bc54a6d4 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -66,10 +66,19 @@ use datafusion_physical_plan::projection::{ProjectionExec, ProjectionExpr}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{ - get_plan_string, DisplayAs, DisplayFormatType, ExecutionPlanProperties, + displayable, get_plan_string, DisplayAs, DisplayFormatType, ExecutionPlanProperties, PlanProperties, Statistics, }; +macro_rules! assert_plan { + ($plan: expr, @ $expected:literal) => { + insta::assert_snapshot!( + displayable($plan.as_ref()).indent(true).to_string(), + @ $expected + ) + }; +} + /// Models operators like BoundedWindowExec that require an input /// ordering but is easy to construct #[derive(Debug)] @@ -3023,12 +3032,15 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> .into(); let plan_parquet = sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); - let expected = &[ - "SortPreservingMergeExec: [c2@1 ASC]", - " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - plans_matches_expected!(expected, &plan_parquet); + + assert_plan!( + plan_parquet, + @r" + SortPreservingMergeExec: [c2@1 ASC] + ProjectionExec: expr=[a@0 as a2, c@2 as c2] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + " + ); // Expected Outcome: // data should not be repartitioned / resorted From 2592835d8d55827763686ea5216d83bb464a4498 Mon Sep 17 00:00:00 2001 From: blaginin Date: Thu, 16 Oct 2025 11:17:19 +0100 Subject: [PATCH 02/41] Move `multi_hash_joins` --- .../enforce_distribution.rs | 266 ++++++++++++------ 1 file changed, 185 insertions(+), 81 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index ba041bc54a6d4..1e70e92ab4903 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -69,6 +69,7 @@ use datafusion_physical_plan::{ displayable, get_plan_string, DisplayAs, DisplayFormatType, ExecutionPlanProperties, PlanProperties, Statistics, }; +use insta::Settings; macro_rules! assert_plan { ($plan: expr, @ $expected:literal) => { @@ -77,6 +78,11 @@ macro_rules! assert_plan { @ $expected ) }; + ($plan: expr, $another_plan: expr) => { + let plan1 = displayable($plan.as_ref()).indent(true).to_string(); + let plan2 = displayable($another_plan.as_ref()).indent(true).to_string(); + assert_eq!(plan1, plan2); + } } /// Models operators like BoundedWindowExec that require an input @@ -526,6 +532,79 @@ impl TestConfig { Ok(optimized) } + + /// Perform a series of runs using the current [`TestConfig`], + /// assert the expected plan result, + /// and return the result plan (for potential subsequent runs). + fn try_run2( + &self, + plan: Arc, + optimizers_to_run: &[Run], + ) -> Result> { + // Add the ancillary output requirements operator at the start: + let optimizer = OutputRequirements::new_add_mode(); + let mut optimized = optimizer.optimize(plan.clone(), &self.config)?; + + // This file has 2 rules that use tree node, apply these rules to original plan consecutively + // After these operations tree nodes should be in a consistent state. + // This code block makes sure that these rules doesn't violate tree node integrity. + { + let adjusted = if self.config.optimizer.top_down_join_key_reordering { + // Run adjust_input_keys_ordering rule + let plan_requirements = + PlanWithKeyRequirements::new_default(plan.clone()); + let adjusted = plan_requirements + .transform_down(adjust_input_keys_ordering) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + adjusted.plan + } else { + // Run reorder_join_keys_to_inputs rule + plan.clone() + .transform_up(|plan| { + Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) + }) + .data()? + }; + + // Then run ensure_distribution rule + DistributionContext::new_default(adjusted) + .transform_up(|distribution_context| { + ensure_distribution(distribution_context, &self.config) + }) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + } + + for run in optimizers_to_run { + optimized = match run { + Run::Distribution => { + let optimizer = EnforceDistribution::new(); + optimizer.optimize(optimized, &self.config)? + } + Run::Sorting => { + let optimizer = EnforceSorting::new(); + optimizer.optimize(optimized, &self.config)? + } + }; + } + + // Remove the ancillary output requirements operator when done: + let optimizer = OutputRequirements::new_remove_mode(); + let optimized = optimizer.optimize(optimized, &self.config)?; + + Ok(optimized) + } + + fn run2( + &self, + plan: Arc, + optimizers_to_run: &[Run], + ) -> Arc { + self.try_run2(plan, optimizers_to_run).unwrap() + } } macro_rules! assert_plan_txt { @@ -565,6 +644,8 @@ fn multi_hash_joins() -> Result<()> { JoinType::RightAnti, ]; + let settings = Settings::clone_current(); + // Join on (a == b1) let join_on = vec![( Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, @@ -573,11 +654,17 @@ fn multi_hash_joins() -> Result<()> { for join_type in join_types { let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = |shift| -> String { - format!("{}HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]", " ".repeat(shift)) - }; - let join_plan_indent2 = join_plan(2); - let join_plan_indent4 = join_plan(4); + + let mut settings = settings.clone(); + settings.add_filter( + // join_type={} replace with join_type=... to avoid snapshot name issue + format!("join_type={join_type}").as_str(), + "join_type=...", + ); + + insta::allow_duplicates! { + settings.bind( || { + match join_type { JoinType::Inner @@ -598,50 +685,58 @@ fn multi_hash_joins() -> Result<()> { &top_join_on, &join_type, ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]"); - let expected = match join_type { + let test_config = TestConfig::default(); + let plan_distrub = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { // Should include 3 RepartitionExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { + + assert_plan!(plan_distrub, @r" + HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, c@2)] + HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + "); + }, // Should include 4 RepartitionExecs - _ => vec![ - top_join_plan.as_str(), - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - &join_plan_indent4, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], + _ => { + assert_plan!(plan_distrub, @r" + HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, c@2)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + "); + }, }; - let test_config = TestConfig::default(); - test_config.run(&expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(&expected, top_join, &SORT_DISTRIB_DISTRIB)?; + + let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrub, plan_sort); } JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => {} } + + match join_type { JoinType::Inner | JoinType::Left @@ -659,55 +754,64 @@ fn multi_hash_joins() -> Result<()> { let top_join = hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); - let top_join_plan = match join_type { - JoinType::RightSemi | JoinType::RightAnti => - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)]"), - _ => - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)]"), - }; - let expected = match join_type { + let test_config = TestConfig::default(); + let plan_distrub = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { // Should include 3 RepartitionExecs - JoinType::Inner | JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => - vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], + JoinType::Inner | JoinType::Right => { + assert_plan!(parquet_exec(), @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet"); + }, + // Should include 3 RepartitionExecs but have a different "on" + JoinType::RightSemi | JoinType::RightAnti => { + assert_plan!(plan_distrub, @r" + HashJoinExec: mode=Partitioned, join_type=..., on=[(b1@1, c@2)] + HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + "); + + } + // Should include 4 RepartitionExecs - _ => - vec![ - top_join_plan.as_str(), - " RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", - &join_plan_indent4, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], + _ => { + assert_plan!(plan_distrub, @r" + HashJoinExec: mode=Partitioned, join_type=..., on=[(b1@6, c@2)] + RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 + HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + "); + + }, }; - let test_config = TestConfig::default(); - test_config.run(&expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(&expected, top_join, &SORT_DISTRIB_DISTRIB)?; + + let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrub, plan_sort); } JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} } + + }); + } } Ok(()) From b5f6160a4749523de3beb958b4ab5bc990c20009 Mon Sep 17 00:00:00 2001 From: blaginin Date: Thu, 16 Oct 2025 11:25:03 +0100 Subject: [PATCH 03/41] Migrate multi_joins_after_alias test to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Replace old-style expected string arrays with insta snapshot assertions. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 72 ++++++++++--------- 1 file changed, 40 insertions(+), 32 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 1e70e92ab4903..6137946db12c9 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -850,23 +850,27 @@ fn multi_joins_after_alias() -> Result<()> { ); // Output partition need to respect the Alias and should not introduce additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)]", - " ProjectionExec: expr=[a@0 as a1, a@0 as a2]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, top_join, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)] + ProjectionExec: expr=[a@0 as a1, a@0 as a2] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); // Join on (a2 == c) let top_join_on = vec![( @@ -877,23 +881,27 @@ fn multi_joins_after_alias() -> Result<()> { let top_join = hash_join_exec(projection, right, &top_join_on, &JoinType::Inner); // Output partition need to respect the Alias and should not introduce additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)]", - " ProjectionExec: expr=[a@0 as a1, a@0 as a2]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, top_join, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)] + ProjectionExec: expr=[a@0 as a1, a@0 as a2] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From 406f590fba059bc3ab60fbe616b0eed1cdcad7af Mon Sep 17 00:00:00 2001 From: blaginin Date: Thu, 16 Oct 2025 11:25:50 +0100 Subject: [PATCH 04/41] Migrate multi_joins_after_multi_alias test to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Replace old-style expected string arrays with insta snapshot assertions. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 41 ++++++++++--------- 1 file changed, 22 insertions(+), 19 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 6137946db12c9..bb147522737c7 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -937,26 +937,29 @@ fn multi_joins_after_multi_alias() -> Result<()> { // The Column 'a' has different meaning now after the two Projections // The original Output partition can not satisfy the Join requirements and need to add an additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " ProjectionExec: expr=[c1@0 as a]", - " ProjectionExec: expr=[c@2 as c1]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, top_join, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + ProjectionExec: expr=[c1@0 as a] + ProjectionExec: expr=[c@2 as c1] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From 99fc2b0c900ba5293eaaebead2304222553eb07c Mon Sep 17 00:00:00 2001 From: blaginin Date: Thu, 16 Oct 2025 11:26:30 +0100 Subject: [PATCH 05/41] Migrate join_after_agg_alias test to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Replace old-style expected string arrays with insta snapshot assertions. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index bb147522737c7..9ec2e6148fa42 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -985,22 +985,26 @@ fn join_after_agg_alias() -> Result<()> { let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); // Only two RepartitionExecs added - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - " RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, join.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, join, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(join.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)] + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] + RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] + RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From d114351f8c3de666e2d3243ae1bcd4344d3e738b Mon Sep 17 00:00:00 2001 From: blaginin Date: Thu, 16 Oct 2025 11:27:10 +0100 Subject: [PATCH 06/41] Migrate hash_join_key_ordering test to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Replace old-style expected string arrays with insta snapshot assertions. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 36 ++++++++++--------- 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 9ec2e6148fa42..e752e109276ca 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1042,23 +1042,27 @@ fn hash_join_key_ordering() -> Result<()> { let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); // Only two RepartitionExecs added - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)]", - " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, join.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, join, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(join.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)] + ProjectionExec: expr=[a1@1 as a1, b1@0 as b1] + AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[] + RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[] + RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From 5704e73aac57ec098266f933ca5724e378cc3b9e Mon Sep 17 00:00:00 2001 From: blaginin Date: Thu, 16 Oct 2025 11:27:58 +0100 Subject: [PATCH 07/41] Migrate multi_hash_join_key_ordering test to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Replace old-style expected string arrays with insta snapshot assertions. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 50 ++++++++++--------- 1 file changed, 27 insertions(+), 23 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index e752e109276ca..80e264a4c8c09 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1166,30 +1166,34 @@ fn multi_hash_join_key_ordering() -> Result<()> { Arc::new(FilterExec::try_new(predicate, top_join)?); // The bottom joins' join key ordering is adjusted based on the top join. And the top join should not introduce additional RepartitionExec - let expected = &[ - "FilterExec: c@6 > 1", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", - " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", - " RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", - " RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, filter_top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, filter_top_join, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(filter_top_join.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + FilterExec: c@6 > 1 + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)] + ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(filter_top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From 6a24a64cf4c823a292f8ae9ef66c231265613893 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:24:02 +0100 Subject: [PATCH 08/41] Migrate multiple tests to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - smj_join_key_ordering - reorder_join_keys_to_left_input (inline snapshots with filter) - parallelization_ignores_limit - parallelization_prior_to_sort_preserving_merge - parallelization_sort_preserving_merge_with_union - parallelization_does_not_benefit Replace old-style expected string arrays with insta inline snapshot assertions. For reorder_join_keys_to_left_input, use regex filter to replace all join_type values with '...' and add separate assertion for top join type. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 232 +++++++++--------- 1 file changed, 116 insertions(+), 116 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 80e264a4c8c09..58e99b3a33cbd 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2883,48 +2883,48 @@ fn parallelization_ignores_limit() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let expected_parquet = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " GlobalLimitExec: skip=0, fetch=100", - " CoalescePartitionsExec", - " LocalLimitExec: fetch=100", - " FilterExec: c@2 = 0", - // repartition should happen prior to the filter to maximize parallelism - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " GlobalLimitExec: skip=0, fetch=100", - // Limit doesn't benefit from input partitioning - no parallelism - " LocalLimitExec: fetch=100", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run( - expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_parquet_distrib, + @r" + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + GlobalLimitExec: skip=0, fetch=100 + CoalescePartitionsExec + LocalLimitExec: fetch=100 + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + GlobalLimitExec: skip=0, fetch=100 + LocalLimitExec: fetch=100 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " GlobalLimitExec: skip=0, fetch=100", - " CoalescePartitionsExec", - " LocalLimitExec: fetch=100", - " FilterExec: c@2 = 0", - // repartition should happen prior to the filter to maximize parallelism - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " GlobalLimitExec: skip=0, fetch=100", - // Limit doesn't benefit from input partitioning - no parallelism - " LocalLimitExec: fetch=100", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_csv_distrib, + @r" + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + GlobalLimitExec: skip=0, fetch=100 + CoalescePartitionsExec + LocalLimitExec: fetch=100 + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + GlobalLimitExec: skip=0, fetch=100 + LocalLimitExec: fetch=100 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false + " + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2991,22 +2991,22 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // parallelization is not beneficial for SortPreservingMerge // Test: with parquet - let expected_parquet = &[ - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run( - expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_parquet_distrib, + @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet" + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = &[ - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - ]; - test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_csv_distrib, + @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false" + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -3036,54 +3036,50 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { // should not sort (as the data was already sorted) // Test: with parquet - let expected_parquet = &[ - "SortPreservingMergeExec: [c@2 ASC]", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run( - expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - let expected_parquet_first_sort_enforcement = &[ - // no SPM - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // has coalesce - " CoalescePartitionsExec", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run( - expected_parquet_first_sort_enforcement, - plan_parquet, - &SORT_DISTRIB_DISTRIB, - )?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_parquet_distrib, + @r" + SortPreservingMergeExec: [c@2 ASC] + UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + " + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_parquet_sort, + @r" + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + " + ); // Test: with csv - let expected_csv = &[ - "SortPreservingMergeExec: [c@2 ASC]", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - ]; - test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - let expected_csv_first_sort_enforcement = &[ - // no SPM - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // has coalesce - " CoalescePartitionsExec", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - ]; - test_config.run( - expected_csv_first_sort_enforcement, - plan_csv.clone(), - &SORT_DISTRIB_DISTRIB, - )?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_csv_distrib, + @r" + SortPreservingMergeExec: [c@2 ASC] + UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false + " + ); + let plan_csv_sort = test_config.run2(plan_csv.clone(), &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_csv_sort, + @r" + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false + " + ); Ok(()) } @@ -3111,24 +3107,28 @@ fn parallelization_does_not_benefit() -> Result<()> { // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism // Test: with parquet - let expected_parquet = &[ - "SortRequiredExec: [c@2 ASC]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run( - expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_parquet_distrib, + @r" + SortRequiredExec: [c@2 ASC] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + " + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = &[ - "SortRequiredExec: [c@2 ASC]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - ]; - test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_csv_distrib, + @r" + SortRequiredExec: [c@2 ASC] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false + " + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } From 80b964f01ef70e1924b6418f23806385f24abba7 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:35:41 +0100 Subject: [PATCH 09/41] Migrate more tests --- .../enforce_distribution.rs | 154 ++++++++++-------- 1 file changed, 84 insertions(+), 70 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 58e99b3a33cbd..ae6809074b02c 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1926,25 +1926,28 @@ fn union_to_interleave() -> Result<()> { aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); // Only two RepartitionExecs added, no final RepartitionExec required - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - " AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", - " InterleaveExec", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] + AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[] + InterleaveExec + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] + RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] + RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1970,28 +1973,31 @@ fn union_not_to_interleave() -> Result<()> { aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); // Only two RepartitionExecs added, no final RepartitionExec required - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - " RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20", - " AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", - " UnionExec", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - // TestConfig: Prefer existing union. let test_config = TestConfig::default().with_prefer_existing_union(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] + RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20 + AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[] + UnionExec + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] + RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[] + RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2001,17 +2007,20 @@ fn added_repartition_to_single_partition() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan = aggregate_exec_with_alias(parquet_exec(), alias); - let expected = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(&expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(&expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2021,18 +2030,21 @@ fn repartition_deepest_node() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2041,19 +2053,21 @@ fn repartition_deepest_node() -> Result<()> { fn repartition_unsorted_limit() -> Result<()> { let plan = limit_exec(filter_exec(parquet_exec())); - let expected = &[ - "GlobalLimitExec: skip=0, fetch=100", - " CoalescePartitionsExec", - " LocalLimitExec: fetch=100", - " FilterExec: c@2 = 0", - // nothing sorts the data, so the local limit doesn't require sorted data either - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" + GlobalLimitExec: skip=0, fetch=100 + CoalescePartitionsExec + LocalLimitExec: fetch=100 + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From a0b4529449b9354d0212441d26bc6f02f929390e Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:41:00 +0100 Subject: [PATCH 10/41] Migrate 11 enforce_distribution tests to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Migrated the following tests to use inline insta snapshots: - repartition_sorted_limit - repartition_sorted_limit_with_filter - repartition_ignores_limit - repartition_ignores_union - repartition_through_sort_preserving_merge - repartition_ignores_sort_preserving_merge - repartition_ignores_sort_preserving_merge_with_union - repartition_does_not_destroy_sort - repartition_does_not_destroy_sort_more_complex - repartition_ignores_transitively_with_projection - repartition_transitively_past_sort_with_projection All tests pass without --accept flag. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 292 ++++++++++-------- 1 file changed, 155 insertions(+), 137 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index ae6809074b02c..01f4a40ee2410 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2082,17 +2082,19 @@ fn repartition_sorted_limit() -> Result<()> { .into(); let plan = limit_exec(sort_exec(sort_key, parquet_exec())); - let expected = &[ - "GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +GlobalLimitExec: skip=0, fetch=100 + LocalLimitExec: fetch=100 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2110,19 +2112,20 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { sort_key, ); - let expected = &[ - "SortRequiredExec: [c@2 ASC]", - " FilterExec: c@2 = 0", - // We can use repartition here, ordering requirement by SortRequiredExec - // is still satisfied. - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortRequiredExec: [c@2 ASC] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2135,26 +2138,27 @@ fn repartition_ignores_limit() -> Result<()> { alias, ); - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " GlobalLimitExec: skip=0, fetch=100", - " CoalescePartitionsExec", - " LocalLimitExec: fetch=100", - " FilterExec: c@2 = 0", - // repartition should happen prior to the filter to maximize parallelism - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - // Expect no repartition to happen for local limit - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + GlobalLimitExec: skip=0, fetch=100 + CoalescePartitionsExec + LocalLimitExec: fetch=100 + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + GlobalLimitExec: skip=0, fetch=100 + LocalLimitExec: fetch=100 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2163,19 +2167,21 @@ fn repartition_ignores_limit() -> Result<()> { fn repartition_ignores_union() -> Result<()> { let plan = union_exec(vec![parquet_exec(); 5]); - let expected = &[ - "UnionExec", - // Expect no repartition of DataSourceExec - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2191,15 +2197,17 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { .into(); let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); - // need resort as the data was not sorted correctly - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2218,24 +2226,26 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { parquet_exec_multiple_sorted(vec![sort_key]), ); - // Test: run EnforceDistribution, then EnforceSort - // - // should not sort (as the data was already sorted) - // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge - let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [c@2 ASC] + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); Ok(()) } @@ -2255,27 +2265,30 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { ]); let plan = sort_preserving_merge_exec(sort_key, input); - // Test: run EnforceDistribution, then EnforceSort. - // - // should not repartition / sort (as the data was already sorted) - let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [c@2 ASC] + UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); // test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); Ok(()) } @@ -2299,16 +2312,18 @@ fn repartition_does_not_destroy_sort() -> Result<()> { // TestConfig: Prefer existing sort. let test_config = TestConfig::default().with_prefer_existing_sort(); - // during repartitioning ordering is preserved - let expected = &[ - "SortRequiredExec: [d@3 ASC]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", - ]; - - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortRequiredExec: [d@3 ASC] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2337,22 +2352,21 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { let input2 = filter_exec(parquet_exec()); let plan = union_exec(vec![input1, input2]); - // should not repartition below the SortRequired as that - // branch doesn't benefit from increased parallelism - let expected = &[ - "UnionExec", - // union input 1: no repartitioning - " SortRequiredExec: [c@2 ASC]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - // union input 2: should repartition - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +UnionExec + SortRequiredExec: [c@2 ASC] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2425,16 +2439,18 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { sort_key, ); - let expected = &[ - "SortRequiredExec: [c@2 ASC]", - // Since this projection is trivial, increasing parallelism is not beneficial - " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortRequiredExec: [c@2 ASC] + ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2460,16 +2476,18 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { ), ); - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // Since this projection is trivial, increasing parallelism is not beneficial - " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From b984753eca099c7c16acaa5a2aa2cfeac3731435 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:44:05 +0100 Subject: [PATCH 11/41] Migrate 8 more enforce_distribution tests to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Migrated the following tests to use inline insta snapshots: - remove_redundant_roundrobins - remove_unnecessary_spm_after_filter - preserve_ordering_through_repartition - no_need_for_sort_after_filter - do_not_preserve_ordering_through_repartition3 - do_not_add_unnecessary_hash - do_not_add_unnecessary_hash2 - optimize_away_unnecessary_repartition All tests pass without --accept flag. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 163 ++++++++++-------- 1 file changed, 93 insertions(+), 70 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 01f4a40ee2410..5c1aac966fabe 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3276,15 +3276,18 @@ fn remove_redundant_roundrobins() -> Result<()> { ]; plans_matches_expected!(expected, &physical_plan); - let expected = &[ - "FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3304,18 +3307,18 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { // TestConfig: Prefer existing sort. let test_config = TestConfig::default().with_prefer_existing_sort(); - // Expected Outcome: - // Original plan expects its output to be ordered by c@2 ASC. - // This is still satisfied since, after filter that column is constant. - let expected = &[ - "CoalescePartitionsExec", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +CoalescePartitionsExec + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3335,14 +3338,18 @@ fn preserve_ordering_through_repartition() -> Result<()> { // TestConfig: Prefer existing sort. let test_config = TestConfig::default().with_prefer_existing_sort(); - let expected = &[ - "SortPreservingMergeExec: [d@3 ASC]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", - ]; - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [d@3 ASC] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3398,17 +3405,19 @@ fn no_need_for_sort_after_filter() -> Result<()> { let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - let expected = &[ - // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. - "CoalescePartitionsExec", - // Since after this stage c is constant. c@2 ASC ordering is already satisfied. - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +CoalescePartitionsExec + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3471,14 +3480,18 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let input = parquet_exec_multiple_sorted(vec![sort_key]); let physical_plan = filter_exec(input); - let expected = &[ - "FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3575,13 +3588,17 @@ fn do_not_add_unnecessary_hash() -> Result<()> { // Make sure target partition number is 1. In this case hash repartition is unnecessary. let test_config = TestConfig::default().with_query_execution_partitions(1); - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3603,19 +3620,21 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { // Make sure target partition number is larger than 2 (e.g partition number at the source). let test_config = TestConfig::default().with_query_execution_partitions(4); - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - // Since hash requirements of this operator is satisfied. There shouldn't be - // a hash repartition here - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3630,12 +3649,16 @@ fn optimize_away_unnecessary_repartition() -> Result<()> { ]; plans_matches_expected!(expected, physical_plan.clone()); - let expected = - &["DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet"]; - let test_config = TestConfig::default(); - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From e095da3ef67f0011eddf050020225e63b77bba0d Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:48:17 +0100 Subject: [PATCH 12/41] Migrate 2 more enforce_distribution tests to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Migrated the following tests to use inline insta snapshots: - optimize_away_unnecessary_repartition2 - parallelization_ignores_transitively_with_projection_csv All tests pass without --accept flag. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 38 +++++++++++-------- 1 file changed, 22 insertions(+), 16 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5c1aac966fabe..bf34c6dfdc345 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3249,15 +3249,17 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { ]; plans_matches_expected!(expected, &plan_csv); - // Expected Outcome: - // data should not be repartitioned / resorted - let expected_csv = &[ - "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - ]; let test_config = TestConfig::default(); - test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +ProjectionExec: expr=[a@0 as a2, c@2 as c2] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false +" + ); + let plan_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3678,15 +3680,19 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { ]; plans_matches_expected!(expected, physical_plan.clone()); - let expected = &[ - "FilterExec: c@2 = 0", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +FilterExec: c@2 = 0 + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } From 221f5421955dac3f080c3feab2aaef7a280fcf50 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:50:52 +0100 Subject: [PATCH 13/41] Migrate 2 more enforce_distribution tests to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Migrated the following tests to use inline insta snapshots: - parallelization_single_partition - parallelization_multiple_files All tests pass without --accept flag. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 96 +++++++++---------- 1 file changed, 46 insertions(+), 50 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index bf34c6dfdc345..0ecdce78b86c2 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2588,28 +2588,32 @@ fn parallelization_single_partition() -> Result<()> { .with_query_execution_partitions(2); // Test: with parquet - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run( - &expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(&expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_parquet_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - test_config.run(&expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(&expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_csv_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +" + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2633,40 +2637,32 @@ fn parallelization_multiple_files() -> Result<()> { // The groups must have only contiguous ranges of rows from the same file // if any group has rows from multiple files, the data is no longer sorted destroyed // https://github.com/apache/datafusion/issues/8451 - let expected_with_3_target_partitions = [ - "SortRequiredExec: [a@0 ASC]", - " FilterExec: c@2 = 0", - " DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; let test_config_concurrency_3 = test_config.clone().with_query_execution_partitions(3); - test_config_concurrency_3.run( - &expected_with_3_target_partitions, - plan.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config_concurrency_3.run( - &expected_with_3_target_partitions, - plan.clone(), - &SORT_DISTRIB_DISTRIB, - )?; + let plan_3_distrib = test_config_concurrency_3.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_3_distrib, + @r" +SortRequiredExec: [a@0 ASC] + FilterExec: c@2 = 0 + DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet +" + ); + let plan_3_sort = test_config_concurrency_3.run2(plan.clone(), &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_3_distrib, plan_3_sort); - let expected_with_8_target_partitions = [ - "SortRequiredExec: [a@0 ASC]", - " FilterExec: c@2 = 0", - " DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; let test_config_concurrency_8 = test_config.with_query_execution_partitions(8); - test_config_concurrency_8.run( - &expected_with_8_target_partitions, - plan.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config_concurrency_8.run( - &expected_with_8_target_partitions, - plan, - &SORT_DISTRIB_DISTRIB, - )?; + let plan_8_distrib = test_config_concurrency_8.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_8_distrib, + @r" +SortRequiredExec: [a@0 ASC] + FilterExec: c@2 = 0 + DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet +" + ); + let plan_8_sort = test_config_concurrency_8.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_8_distrib, plan_8_sort); Ok(()) } From 5486ad7b37698ab09c8fc46a66ca1e125d683cd6 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:55:37 +0100 Subject: [PATCH 14/41] Migrate 3 more enforce_distribution tests with different optimizer outputs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Migrated the following tests to use inline insta snapshots: - repartition_transitively_with_projection - repartition_transitively_past_sort_with_filter - repartition_transitively_past_sort_with_projection_and_filter These tests have different outputs for DISTRIB_DISTRIB_SORT vs SORT_DISTRIB_DISTRIB, so each gets two separate inline snapshots. All tests pass without --accept flag. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 127 ++++++++++-------- 1 file changed, 68 insertions(+), 59 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 0ecdce78b86c2..81655d3737dcf 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2391,28 +2391,31 @@ fn repartition_transitively_with_projection() -> Result<()> { .into(); let plan = sort_preserving_merge_exec(sort_key, proj); - // Test: run EnforceDistribution, then EnforceSort. - let expected = &[ - "SortPreservingMergeExec: [sum@0 ASC]", - " SortExec: expr=[sum@0 ASC], preserve_partitioning=[true]", - // Since this projection is not trivial, increasing parallelism is beneficial - " ProjectionExec: expr=[a@0 + b@1 as sum]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [sum@0 ASC] + SortExec: expr=[sum@0 ASC], preserve_partitioning=[true] + ProjectionExec: expr=[a@0 + b@1 as sum] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[sum@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - // Since this projection is not trivial, increasing parallelism is beneficial - " ProjectionExec: expr=[a@0 + b@1 as sum]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortExec: expr=[sum@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + ProjectionExec: expr=[a@0 + b@1 as sum] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); Ok(()) } @@ -2502,28 +2505,31 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { .into(); let plan = sort_exec(sort_key, filter_exec(parquet_exec())); - // Test: run EnforceDistribution, then EnforceSort. - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [a@0 ASC] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " FilterExec: c@2 = 0", - // Expect repartition on the input of the filter (as it can benefit from additional parallelism) - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); Ok(()) } @@ -2549,30 +2555,33 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> ), ); - // Test: run EnforceDistribution, then EnforceSort. - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - " FilterExec: c@2 = 0", - // repartition is lowest down - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [a@0 ASC] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); Ok(()) } From 9fd6982a620c20d0e5ae130c758d54b98ec25285 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:57:35 +0100 Subject: [PATCH 15/41] More --- .../enforce_distribution.rs | 88 ++++++++++--------- 1 file changed, 45 insertions(+), 43 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 81655d3737dcf..5dffb270b7230 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3374,29 +3374,30 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let test_config = TestConfig::default(); - // Test: run EnforceDistribution, then EnforceSort. - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [a@0 ASC] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet +" + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; - test_config.run( - expected_first_sort_enforcement, - physical_plan, - &SORT_DISTRIB_DISTRIB, - )?; + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet +" + ); Ok(()) } @@ -3448,30 +3449,31 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let test_config = TestConfig::default(); - // Test: run EnforceDistribution, then EnforceSort. - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [a@0 ASC] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - test_config.run( - expected_first_sort_enforcement, - physical_plan, - &SORT_DISTRIB_DISTRIB, - )?; + let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); Ok(()) } From f70f50e31ecb03a3154d53c916a0425eab116779 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:59:07 +0100 Subject: [PATCH 16/41] Migrate test_distribute_sort_parquet to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Migrated test_distribute_sort_parquet to use inline insta snapshots. This test uses different optimizer configurations: - Initial plan before optimization - After Run::Distribution - After Run::Distribution + Run::Sorting All three states now use inline snapshots instead of string arrays. All tests pass without --accept flag. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 42 +++++++++++-------- 1 file changed, 25 insertions(+), 17 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5dffb270b7230..4f7e3438804a0 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3719,27 +3719,35 @@ async fn test_distribute_sort_parquet() -> Result<()> { let physical_plan = sort_exec(sort_key, parquet_exec_with_stats(10000 * 8192)); // prior to optimization, this is the starting plan - let starting = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - plans_matches_expected!(starting, physical_plan.clone()); + assert_plan!( + physical_plan, + @r" +SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); // what the enforce distribution run does. - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run(expected, physical_plan.clone(), &[Run::Distribution])?; + let plan_distribution = test_config.run2(physical_plan.clone(), &[Run::Distribution]); + assert_plan!( + plan_distribution, + @r" +SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); // what the sort parallelization (in enforce sorting), does after the enforce distribution changes - let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run(expected, physical_plan, &[Run::Distribution, Run::Sorting])?; + let plan_both = test_config.run2(physical_plan, &[Run::Distribution, Run::Sorting]); + assert_plan!( + plan_both, + @r" +SortPreservingMergeExec: [c@2 ASC] + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); Ok(()) } From 8a363aad8094771d301d8072f5eedbb3a4df2f6a Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 19:59:55 +0100 Subject: [PATCH 17/41] Migrate test_distribute_sort_memtable to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Migrated test_distribute_sort_memtable to use inline insta snapshot. This test checks the final optimized plan after creating and executing a SQL query on a MemTable. All tests pass without --accept flag. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../physical_optimizer/enforce_distribution.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 4f7e3438804a0..d25f5f7c62e46 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3772,12 +3772,14 @@ async fn test_distribute_sort_memtable() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; // this is the final, optimized plan - let expected = &[ - "SortPreservingMergeExec: [id@0 ASC NULLS LAST]", - " SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true]", - " DataSourceExec: partitions=3, partition_sizes=[34, 33, 33]", - ]; - plans_matches_expected!(expected, physical_plan); + assert_plan!( + physical_plan, + @r" +SortPreservingMergeExec: [id@0 ASC NULLS LAST] + SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] + DataSourceExec: partitions=3, partition_sizes=[34, 33, 33] +" + ); Ok(()) } From a672a0077a71ebf3672df2ebc5d3d1d0ea456077 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:06:00 +0100 Subject: [PATCH 18/41] Migrate optimize_away_unnecessary_repartition2 to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index d25f5f7c62e46..d3010f8a90479 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3677,15 +3677,17 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( filter_exec(repartition_exec(parquet_exec())), ))); - let expected = &[ - "FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CoalescePartitionsExec", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - plans_matches_expected!(expected, physical_plan.clone()); + assert_plan!( + physical_plan, + @r" +FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + CoalescePartitionsExec + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); From 54a2b40aface96cd42cd969a0e1d0fb7c283d8aa Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:06:57 +0100 Subject: [PATCH 19/41] Migrate parallelization_ignores_transitively_with_projection_csv to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../physical_optimizer/enforce_distribution.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index d3010f8a90479..1889e877af999 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3247,12 +3247,14 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { }] .into(); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); - let expected = &[ - "SortPreservingMergeExec: [c2@1 ASC]", - " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - ]; - plans_matches_expected!(expected, &plan_csv); + assert_plan!( + plan_csv, + @r" +SortPreservingMergeExec: [c2@1 ASC] + ProjectionExec: expr=[a@0 as a2, c@2 as c2] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false +" + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); From a9783e3382620ab27167402cd14da764c3e4a380 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:07:54 +0100 Subject: [PATCH 20/41] Migrate remove_redundant_roundrobins to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../physical_optimizer/enforce_distribution.rs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 1889e877af999..e87db8d170411 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3276,14 +3276,16 @@ fn remove_redundant_roundrobins() -> Result<()> { let input = parquet_exec(); let repartition = repartition_exec(repartition_exec(input)); let physical_plan = repartition_exec(filter_exec(repartition)); - let expected = &[ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - plans_matches_expected!(expected, &physical_plan); + assert_plan!( + physical_plan, + @r" +RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); From 9b73d3545acf800bccf11c027956795a6624e0bc Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:08:49 +0100 Subject: [PATCH 21/41] Migrate optimize_away_unnecessary_repartition to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../physical_optimizer/enforce_distribution.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index e87db8d170411..67c49c7d0b0f4 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3655,12 +3655,14 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] #[test] fn optimize_away_unnecessary_repartition() -> Result<()> { let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); - let expected = &[ - "CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - plans_matches_expected!(expected, physical_plan.clone()); + assert_plan!( + physical_plan, + @r" +CoalescePartitionsExec + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); From 6e9dab7ad9a242f40f28f3e1a0c77113d20eb757 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:09:32 +0100 Subject: [PATCH 22/41] Kill `plans_matches_expected` --- .../enforce_distribution.rs | 307 +++++++++--------- 1 file changed, 149 insertions(+), 158 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 67c49c7d0b0f4..9bce22ae8a3f1 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -367,22 +367,6 @@ fn ensure_distribution_helper( ensure_distribution(distribution_context, &config).map(|item| item.data.plan) } -/// Test whether plan matches with expected plan -macro_rules! plans_matches_expected { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let physical_plan = $PLAN; - let actual = get_plan_string(&physical_plan); - - let expected_plan_lines: Vec<&str> = $EXPECTED_LINES - .iter().map(|s| *s).collect(); - - assert_eq!( - expected_plan_lines, actual, - "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" - ); - } -} - fn test_suite_default_config_options() -> ConfigOptions { let mut config = ConfigOptions::new(); @@ -2085,14 +2069,14 @@ fn repartition_sorted_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2115,15 +2099,15 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [c@2 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2141,8 +2125,8 @@ fn repartition_ignores_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] @@ -2156,7 +2140,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] LocalLimitExec: fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2170,8 +2154,8 @@ fn repartition_ignores_union() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet @@ -2179,7 +2163,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2200,12 +2184,12 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2229,23 +2213,23 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [c@2 ASC] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -2268,27 +2252,27 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [c@2 ASC] UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -2314,14 +2298,14 @@ fn repartition_does_not_destroy_sort() -> Result<()> { let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [d@3 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2355,8 +2339,8 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" UnionExec SortRequiredExec: [c@2 ASC] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet @@ -2364,7 +2348,7 @@ UnionExec RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2394,28 +2378,28 @@ fn repartition_transitively_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [sum@0 ASC] SortExec: expr=[sum@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a@0 + b@1 as sum] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[sum@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec ProjectionExec: expr=[a@0 + b@1 as sum] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2445,13 +2429,13 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [c@2 ASC] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2482,13 +2466,13 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2508,28 +2492,28 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2558,8 +2542,8 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] @@ -2567,13 +2551,13 @@ SortPreservingMergeExec: [a@0 ASC] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] @@ -2581,7 +2565,7 @@ SortExec: expr=[a@0 ASC], preserve_partitioning=[false] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2597,30 +2581,31 @@ fn parallelization_single_partition() -> Result<()> { .with_query_execution_partitions(2); // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2648,28 +2633,30 @@ fn parallelization_multiple_files() -> Result<()> { // https://github.com/apache/datafusion/issues/8451 let test_config_concurrency_3 = test_config.clone().with_query_execution_partitions(3); - let plan_3_distrib = test_config_concurrency_3.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_3_distrib = + test_config_concurrency_3.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_3_distrib, - @r" + plan_3_distrib, + @r" SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); let plan_3_sort = test_config_concurrency_3.run2(plan.clone(), &SORT_DISTRIB_DISTRIB); assert_plan!(plan_3_distrib, plan_3_sort); let test_config_concurrency_8 = test_config.with_query_execution_partitions(8); - let plan_8_distrib = test_config_concurrency_8.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_8_distrib = + test_config_concurrency_8.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_8_distrib, - @r" + plan_8_distrib, + @r" SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); let plan_8_sort = test_config_concurrency_8.run2(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_8_distrib, plan_8_sort); @@ -2920,7 +2907,8 @@ fn parallelization_ignores_limit() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @r" @@ -3028,7 +3016,8 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // parallelization is not beneficial for SortPreservingMerge // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet" @@ -3073,7 +3062,8 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { // should not sort (as the data was already sorted) // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @r" @@ -3144,7 +3134,8 @@ fn parallelization_does_not_benefit() -> Result<()> { // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @r" @@ -3248,23 +3239,23 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { .into(); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); assert_plan!( - plan_csv, - @r" + plan_csv, + @r" SortPreservingMergeExec: [c2@1 ASC] ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " - ); + ); let plan_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3277,26 +3268,26 @@ fn remove_redundant_roundrobins() -> Result<()> { let repartition = repartition_exec(repartition_exec(input)); let physical_plan = repartition_exec(filter_exec(repartition)); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3320,14 +3311,14 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3351,14 +3342,14 @@ fn preserve_ordering_through_repartition() -> Result<()> { let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [d@3 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3380,28 +3371,28 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -3420,14 +3411,14 @@ fn no_need_for_sort_after_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3455,21 +3446,21 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -3477,7 +3468,7 @@ SortExec: expr=[a@0 ASC], preserve_partitioning=[false] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -3496,13 +3487,13 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3603,13 +3594,13 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3635,8 +3626,8 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] @@ -3645,7 +3636,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3656,22 +3647,22 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] fn optimize_away_unnecessary_repartition() -> Result<()> { let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" CoalescePartitionsExec RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3684,8 +3675,8 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { filter_exec(repartition_exec(parquet_exec())), ))); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 CoalescePartitionsExec @@ -3693,19 +3684,19 @@ FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3728,34 +3719,34 @@ async fn test_distribute_sort_parquet() -> Result<()> { // prior to optimization, this is the starting plan assert_plan!( - physical_plan, - @r" + physical_plan, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // what the enforce distribution run does. let plan_distribution = test_config.run2(physical_plan.clone(), &[Run::Distribution]); assert_plan!( - plan_distribution, - @r" + plan_distribution, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // what the sort parallelization (in enforce sorting), does after the enforce distribution changes let plan_both = test_config.run2(physical_plan, &[Run::Distribution, Run::Sorting]); assert_plan!( - plan_both, - @r" + plan_both, + @r" SortPreservingMergeExec: [c@2 ASC] SortExec: expr=[c@2 ASC], preserve_partitioning=[true] DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -3781,13 +3772,13 @@ async fn test_distribute_sort_memtable() -> Result<()> { // this is the final, optimized plan assert_plan!( - physical_plan, - @r" + physical_plan, + @r" SortPreservingMergeExec: [id@0 ASC NULLS LAST] SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] DataSourceExec: partitions=3, partition_sizes=[34, 33, 33] " - ); + ); Ok(()) } From 6cec480353f2bf5cfd207626dcae7363a0d71a3c Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:12:17 +0100 Subject: [PATCH 23/41] Migrate merge_does_not_need_sort to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 32 +++++++++++-------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 9bce22ae8a3f1..f7b5f2a3d6ffa 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1864,13 +1864,16 @@ fn merge_does_not_need_sort() -> Result<()> { // // The optimizer should not add an additional SortExec as the // data is already sorted - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - " CoalesceBatchesExec: target_batch_size=4096", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; let test_config = TestConfig::default(); - test_config.run(expected, exec.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(exec.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortPreservingMergeExec: [a@0 ASC] + CoalesceBatchesExec: target_batch_size=4096 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet +" + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: // @@ -1878,13 +1881,16 @@ fn merge_does_not_need_sort() -> Result<()> { // (according to flag: PREFER_EXISTING_SORT) // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with // SortExec at the top. - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=4096", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, exec, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.run2(exec, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + CoalesceBatchesExec: target_batch_size=4096 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet +" + ); Ok(()) } From b00481d922b61386fe490c39834e113fba90a4f3 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:15:54 +0100 Subject: [PATCH 24/41] Migrate parallelization_compressed_csv to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This test uses a loop to test different compression types with conditional logic for expected outputs. Used insta::allow_duplicates! to handle inline snapshots within the loop. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 86 ++++++++++--------- 1 file changed, 47 insertions(+), 39 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index f7b5f2a3d6ffa..a08913dd62c47 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2681,46 +2681,54 @@ fn parallelization_compressed_csv() -> Result<()> { FileCompressionType::UNCOMPRESSED, ]; - let expected_not_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - - let expected_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - - for compression_type in compression_types { - let expected = if compression_type.is_compressed() { - &expected_not_partitioned[..] - } else { - &expected_partitioned[..] - }; + insta::allow_duplicates! { + for compression_type in compression_types { + let plan = aggregate_exec_with_alias( + DataSourceExec::from_data_source( + FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_file_compression_type(compression_type) + .build(), + ), + vec![("a".to_string(), "a".to_string())], + ); + let test_config = TestConfig::default() + .with_query_execution_partitions(2) + .with_prefer_repartition_file_scans(10); + + let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + if compression_type.is_compressed() { + // Compressed files cannot be partitioned + assert_plan!( + plan_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +" + ); + } else { + // Uncompressed files can be partitioned + assert_plan!( + plan_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +" + ); + } - let plan = aggregate_exec_with_alias( - DataSourceExec::from_data_source( - FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_file_compression_type(compression_type) - .build(), - ), - vec![("a".to_string(), "a".to_string())], - ); - let test_config = TestConfig::default() - .with_query_execution_partitions(2) - .with_prefer_repartition_file_scans(10); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); + } } Ok(()) } From 8439cbb6105b32450fdfac7e2102bc16ac52c73c Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:16:41 +0100 Subject: [PATCH 25/41] Migrate parallelization_two_partitions to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 46 ++++++++++--------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index a08913dd62c47..a073ba2d17aea 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2744,30 +2744,32 @@ fn parallelization_two_partitions() -> Result<()> { .with_prefer_repartition_file_scans(10); // Test: with parquet - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Plan already has two partitions - " DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run( - &expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(&expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_parquet_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Plan already has two partitions - " DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - test_config.run(&expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(&expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_csv_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +" + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } From c80a32d4bc9dc6e7d9ad6e7ebc8746a2cb40f6f6 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:18:15 +0100 Subject: [PATCH 26/41] Migrate parallelization_two_partitions_into_four to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 48 ++++++++++--------- 1 file changed, 26 insertions(+), 22 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index a073ba2d17aea..5bdfb00239951 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2785,30 +2785,34 @@ fn parallelization_two_partitions_into_four() -> Result<()> { .with_prefer_repartition_file_scans(10); // Test: with parquet - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Multiple source files split across partitions - " DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run( - &expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(&expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + // Multiple source files split across partitions + assert_plan!( + plan_parquet_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Multiple source files split across partitions - " DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - test_config.run(&expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(&expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + // Multiple source files split across partitions + assert_plan!( + plan_csv_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +" + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } From d6da5ef99e4484ac16318b990f5b3d16407a7b00 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:19:43 +0100 Subject: [PATCH 27/41] Migrate parallelization_sorted_limit to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 52 ++++++++++--------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5bdfb00239951..947b242ab4255 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2831,32 +2831,36 @@ fn parallelization_sorted_limit() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let expected_parquet = &[ - "GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // Doesn't parallelize for SortExec without preserve_partitioning - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run( - expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + // data is sorted so can't repartition here + // Doesn't parallelize for SortExec without preserve_partitioning + assert_plan!( + plan_parquet_distrib, + @r" +GlobalLimitExec: skip=0, fetch=100 + LocalLimitExec: fetch=100 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = &[ - "GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // Doesn't parallelize for SortExec without preserve_partitioning - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + // data is sorted so can't repartition here + // Doesn't parallelize for SortExec without preserve_partitioning + assert_plan!( + plan_csv_distrib, + @r" +GlobalLimitExec: skip=0, fetch=100 + LocalLimitExec: fetch=100 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +" + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } From ceef90f96ae7c42c17e31e1bc86be9d42e5af5b6 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:21:09 +0100 Subject: [PATCH 28/41] Migrate parallelization_limit_with_filter to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 68 ++++++++++--------- 1 file changed, 36 insertions(+), 32 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 947b242ab4255..a4376e722e9e0 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2880,40 +2880,44 @@ fn parallelization_limit_with_filter() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let expected_parquet = &[ - "GlobalLimitExec: skip=0, fetch=100", - " CoalescePartitionsExec", - " LocalLimitExec: fetch=100", - " FilterExec: c@2 = 0", - // even though data is sorted, we can use repartition here. Since - // ordering is not used in subsequent stages anyway. - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // SortExec doesn't benefit from input partitioning - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run( - expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + // even though data is sorted, we can use repartition here. Since + // ordering is not used in subsequent stages anyway. + // SortExec doesn't benefit from input partitioning + assert_plan!( + plan_parquet_distrib, + @r" +GlobalLimitExec: skip=0, fetch=100 + CoalescePartitionsExec + LocalLimitExec: fetch=100 + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = &[ - "GlobalLimitExec: skip=0, fetch=100", - " CoalescePartitionsExec", - " LocalLimitExec: fetch=100", - " FilterExec: c@2 = 0", - // even though data is sorted, we can use repartition here. Since - // ordering is not used in subsequent stages anyway. - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // SortExec doesn't benefit from input partitioning - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + // even though data is sorted, we can use repartition here. Since + // ordering is not used in subsequent stages anyway. + // SortExec doesn't benefit from input partitioning + assert_plan!( + plan_csv_distrib, + @r" +GlobalLimitExec: skip=0, fetch=100 + CoalescePartitionsExec + LocalLimitExec: fetch=100 + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +" + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } From 2a2b86cac1ff93cccc91c3bfc146022867b6f423 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:22:12 +0100 Subject: [PATCH 29/41] Migrate parallelization_ignores_transitively_with_projection_parquet to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index a4376e722e9e0..e9c0b4626d645 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3225,19 +3225,19 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> " ); + let test_config = TestConfig::default(); + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Expected Outcome: // data should not be repartitioned / resorted - let expected_parquet = &[ - "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - ]; - let test_config = TestConfig::default(); - test_config.run( - expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + assert_plan!( + plan_parquet_distrib, + @r" +ProjectionExec: expr=[a@0 as a2, c@2 as c2] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet +" + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); Ok(()) } From 199a84de97f251897a7b28256c81e6f802dee1f9 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:23:13 +0100 Subject: [PATCH 30/41] Migrate parallelization_union_inputs to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 56 ++++++++++--------- 1 file changed, 30 insertions(+), 26 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index e9c0b4626d645..25d01b0ff2ab1 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -2990,34 +2990,38 @@ fn parallelization_union_inputs() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let expected_parquet = &[ - "UnionExec", - // Union doesn't benefit from input partitioning - no parallelism - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run( - expected_parquet, - plan_parquet.clone(), - &DISTRIB_DISTRIB_SORT, - )?; - test_config.run(expected_parquet, plan_parquet, &SORT_DISTRIB_DISTRIB)?; + let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + // Union doesn't benefit from input partitioning - no parallelism + assert_plan!( + plan_parquet_distrib, + @r" +UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let expected_csv = &[ - "UnionExec", - // Union doesn't benefit from input partitioning - no parallelism - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - test_config.run(expected_csv, plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected_csv, plan_csv, &SORT_DISTRIB_DISTRIB)?; + let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + // Union doesn't benefit from input partitioning - no parallelism + assert_plan!( + plan_csv_distrib, + @r" +UnionExec + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +" + ); + let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } From 83e1e52a5ed145717a41bfc8b7480d23cc59daa0 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:24:50 +0100 Subject: [PATCH 31/41] Migrate smj_join_key_ordering to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 94 ++++++++++--------- 1 file changed, 50 insertions(+), 44 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 25d01b0ff2ab1..3bef51befba64 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1790,52 +1790,58 @@ fn smj_join_key_ordering() -> Result<()> { // Test: run EnforceDistribution, then EnforceSort. // Only two RepartitionExecs added - let expected = &[ - "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - " SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", - " ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", - " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", - " ProjectionExec: expr=[a@1 as a2, b@0 as b2]", - " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run(expected, join.clone(), &DISTRIB_DISTRIB_SORT)?; + let plan_distrib = test_config.run2(join.clone(), &DISTRIB_DISTRIB_SORT); + assert_plan!( + plan_distrib, + @r" +SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] + SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true] + ProjectionExec: expr=[a1@0 as a3, b1@1 as b3] + ProjectionExec: expr=[a1@1 as a1, b1@0 as b1] + AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[] + RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true] + ProjectionExec: expr=[a@1 as a2, b@0 as b2] + AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[] + RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let expected_first_sort_enforcement = &[ - "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - " RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", - " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " ProjectionExec: expr=[a@1 as a2, b@0 as b2]", - " AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - " AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - test_config.run(expected_first_sort_enforcement, join, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.run2(join, &SORT_DISTRIB_DISTRIB); + assert_plan!( + plan_sort, + @r" +SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] + RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + ProjectionExec: expr=[a1@0 as a3, b1@1 as b3] + ProjectionExec: expr=[a1@1 as a1, b1@0 as b1] + AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[] + RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + ProjectionExec: expr=[a@1 as a2, b@0 as b2] + AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[] + RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10 + AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); Ok(()) } From eb7e883b8f5e5308a34ca919f3362b1fab533327 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 20:53:20 +0100 Subject: [PATCH 32/41] Migrate reorder_join_keys_to_left_input test to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add hide_first() helper function that replaces the first occurrence of a regex pattern in a plan, returning the captured value and modified string. Use this to test join key reordering across multiple join types while keeping snapshots readable. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 87 ++++++++++++------- 1 file changed, 55 insertions(+), 32 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 3bef51befba64..f72bcdff2380a 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -71,6 +71,27 @@ use datafusion_physical_plan::{ }; use insta::Settings; +/// Helper function to replace only the first occurrence of a regex pattern in a plan +/// Returns (captured_group_1, modified_string) +fn hide_first(plan: &dyn ExecutionPlan, regex: &str, replacement: &str) -> (String, String) { + let plan_str = displayable(plan).indent(true).to_string(); + let pattern = regex::Regex::new(regex).unwrap(); + + if let Some(captures) = pattern.captures(&plan_str) { + let full_match = captures.get(0).unwrap(); + let captured_value = captures.get(1).map(|m| m.as_str().to_string()).unwrap_or_default(); + let pos = full_match.start(); + let end_pos = full_match.end(); + let mut result = String::with_capacity(plan_str.len()); + result.push_str(&plan_str[..pos]); + result.push_str(replacement); + result.push_str(&plan_str[end_pos..]); + (captured_value, result) + } else { + (String::new(), plan_str) + } +} + macro_rules! assert_plan { ($plan: expr, @ $expected:literal) => { insta::assert_snapshot!( @@ -1281,41 +1302,43 @@ fn reorder_join_keys_to_left_input() -> Result<()> { JoinType::RightAnti, ]; - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type); + insta::allow_duplicates! { + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); - let reordered = reorder_join_keys_to_inputs(top_join)?; + let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); - // The top joins' join key ordering is adjusted based on the children inputs. - let expected = &[ - top_join_plan.as_str(), - " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", - " RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", - " RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; + // The top joins' join key ordering is adjusted based on the children inputs. + let (captured_join_type, modified_plan) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); + assert_eq!(captured_join_type, join_type.to_string()); - assert_plan_txt!(expected, reordered); + insta::assert_snapshot!(modified_plan, @r" +HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)] + ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)] + RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } } Ok(()) From 787d0ff4d1406c98948982e9c99a1c9a099d677d Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 21:33:29 +0100 Subject: [PATCH 33/41] Migrate multi_smj_joins test to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Converted the complex multi_smj_joins test to use insta snapshot testing with filters. This test loops over 6 join types and tests multiple configurations with different optimizer runs. Key changes: - Wrapped loop with insta::allow_duplicates! - Added Settings filter to replace join_type={actual_value} with join_type=... - Converted all test_config.run() calls to run2() + assert_plan!() with inline snapshots - Preserved all existing comments explaining test behavior - Removed unused join_plan helper functions and dynamic string variables 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 498 +++++++++--------- 1 file changed, 254 insertions(+), 244 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index f72bcdff2380a..b10abf0bd54b0 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1509,252 +1509,262 @@ fn multi_smj_joins() -> Result<()> { Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, )]; - for join_type in join_types { - let join = - sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = |shift| -> String { - format!( - "{}SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]", - " ".repeat(shift) - ) - }; - let join_plan_indent2 = join_plan(2); - let join_plan_indent6 = join_plan(6); - let join_plan_indent10 = join_plan(10); - - // Top join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); - let top_join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]"); - - let expected = match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => - vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test - // cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => vec![ - top_join_plan.as_str(), - // Below 2 operators are differences introduced, when join mode is changed - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - &join_plan_indent6, - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - }; - // TODO(wiedld): show different test result if enforce sorting first. - test_config.run(&expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - - let expected_first_sort_enforcement = match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => - vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition and Roundrobin repartition after - // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => vec![ - top_join_plan.as_str(), - // Below 4 operators are differences introduced, when join mode is changed - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - &join_plan_indent10, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - }; - // TODO(wiedld): show different test result if enforce distribution first. - test_config.run( - &expected_first_sort_enforcement, - top_join, - &SORT_DISTRIB_DISTRIB, - )?; + insta::allow_duplicates! { + for join_type in join_types { + let join = + sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); + + // Top join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); + + let mut settings = Settings::clone_current(); + settings.add_filter(&format!("join_type={}", join_type), "join_type=..."); + settings.bind(|| { + let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); - let top_join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]"); - - let expected = match join_type { - // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs - JoinType::Inner | JoinType::Right => vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs - JoinType::Left | JoinType::Full => vec![ - top_join_plan.as_str(), - " SortExec: expr=[b1@6 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", - &join_plan_indent6, - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // this match arm cannot be reached - _ => unreachable!() - }; - // TODO(wiedld): show different test result if enforce sorting first. - test_config.run(&expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - - let expected_first_sort_enforcement = match join_type { - // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs - JoinType::Inner | JoinType::Right => vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs - JoinType::Left | JoinType::Full => vec![ - top_join_plan.as_str(), - " RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@6 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - &join_plan_indent10, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // this match arm cannot be reached - _ => unreachable!() - }; + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + assert_plan!(plan_distrib, @r" +SortMergeJoin: join_type=..., on=[(a@0, c@2)] + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test + // cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + assert_plan!(plan_distrib, @r" +SortMergeJoin: join_type=..., on=[(a@0, c@2)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } + } - // TODO(wiedld): show different test result if enforce distribution first. - test_config.run( - &expected_first_sort_enforcement, - top_join, - &SORT_DISTRIB_DISTRIB, - )?; - } - _ => {} + let plan_sort = test_config.run2(top_join.clone(), &SORT_DISTRIB_DISTRIB); + + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" +SortMergeJoin: join_type=..., on=[(a@0, c@2)] + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } + // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition and Roundrobin repartition after + // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" +SortMergeJoin: join_type=..., on=[(a@0, c@2)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } + } + + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); + + let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { + // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" +SortMergeJoin: join_type=..., on=[(b1@6, c@2)] + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" +SortMergeJoin: join_type=..., on=[(b1@6, c@2)] + SortExec: expr=[b1@6 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } + // this match arm cannot be reached + _ => unreachable!() + } + + let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + + match join_type { + // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" +SortMergeJoin: join_type=..., on=[(b1@6, c@2)] + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } + // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" +SortMergeJoin: join_type=..., on=[(b1@6, c@2)] + RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@6 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } + // this match arm cannot be reached + _ => unreachable!() + } + } + _ => {} + } + }); } } From 59e7353c257eb6524f0b12fec9c4a93f420fd07e Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 21:35:32 +0100 Subject: [PATCH 34/41] Migrate reorder_join_keys_to_right_input test to insta snapshots MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Converted the reorder_join_keys_to_right_input test to use insta snapshot testing with the hide_first helper function. This test loops over 8 join types and verifies that join key ordering is adjusted based on children inputs. Key changes: - Wrapped loop with insta::allow_duplicates! - Used hide_first() to replace only the first join_type occurrence with "..." - Converted assert_plan_txt! to insta::assert_snapshot! with inline snapshot - Preserved comment explaining the test behavior - Removed dynamic top_join_plan string variable 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 70 +++++++++---------- 1 file changed, 35 insertions(+), 35 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index b10abf0bd54b0..5de41d128e170 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1438,41 +1438,41 @@ fn reorder_join_keys_to_right_input() -> Result<()> { JoinType::RightAnti, ]; - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)]", &join_type); - - let reordered = reorder_join_keys_to_inputs(top_join)?; - - // The top joins' join key ordering is adjusted based on the children inputs. - let expected = &[ - top_join_plan.as_str(), - " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", - " RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", - " RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - - assert_plan_txt!(expected, reordered); + insta::allow_duplicates! { + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); + + let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); + + // The top joins' join key ordering is adjusted based on the children inputs. + let (_, plan_str) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); + insta::assert_snapshot!(plan_str, @r" +HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)] + ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)] + RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)] + RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); + } } Ok(()) From 5f9ce0aa825d517b5289eb94e83a622374a56f07 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 21:36:09 +0100 Subject: [PATCH 35/41] kill old run --- .../enforce_distribution.rs | 353 +++++++----------- 1 file changed, 139 insertions(+), 214 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5de41d128e170..3319d56e0d553 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -462,86 +462,11 @@ impl TestConfig { self } - /// Perform a series of runs using the current [`TestConfig`], - /// assert the expected plan result, - /// and return the result plan (for potential subsequent runs). - fn run( - &self, - expected_lines: &[&str], - plan: Arc, - optimizers_to_run: &[Run], - ) -> Result> { - let expected_lines: Vec<&str> = expected_lines.to_vec(); - - // Add the ancillary output requirements operator at the start: - let optimizer = OutputRequirements::new_add_mode(); - let mut optimized = optimizer.optimize(plan.clone(), &self.config)?; - - // This file has 2 rules that use tree node, apply these rules to original plan consecutively - // After these operations tree nodes should be in a consistent state. - // This code block makes sure that these rules doesn't violate tree node integrity. - { - let adjusted = if self.config.optimizer.top_down_join_key_reordering { - // Run adjust_input_keys_ordering rule - let plan_requirements = - PlanWithKeyRequirements::new_default(plan.clone()); - let adjusted = plan_requirements - .transform_down(adjust_input_keys_ordering) - .data() - .and_then(check_integrity)?; - // TODO: End state payloads will be checked here. - adjusted.plan - } else { - // Run reorder_join_keys_to_inputs rule - plan.clone() - .transform_up(|plan| { - Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) - }) - .data()? - }; - - // Then run ensure_distribution rule - DistributionContext::new_default(adjusted) - .transform_up(|distribution_context| { - ensure_distribution(distribution_context, &self.config) - }) - .data() - .and_then(check_integrity)?; - // TODO: End state payloads will be checked here. - } - - for run in optimizers_to_run { - optimized = match run { - Run::Distribution => { - let optimizer = EnforceDistribution::new(); - optimizer.optimize(optimized, &self.config)? - } - Run::Sorting => { - let optimizer = EnforceSorting::new(); - optimizer.optimize(optimized, &self.config)? - } - }; - } - - // Remove the ancillary output requirements operator when done: - let optimizer = OutputRequirements::new_remove_mode(); - let optimized = optimizer.optimize(optimized, &self.config)?; - - // Now format correctly - let actual_lines = get_plan_string(&optimized); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{expected_lines:#?}\nactual:\n\n{actual_lines:#?}\n\n" - ); - - Ok(optimized) - } /// Perform a series of runs using the current [`TestConfig`], /// assert the expected plan result, /// and return the result plan (for potential subsequent runs). - fn try_run2( + fn try_run( &self, plan: Arc, optimizers_to_run: &[Run], @@ -603,12 +528,12 @@ impl TestConfig { Ok(optimized) } - fn run2( + fn run( &self, plan: Arc, optimizers_to_run: &[Run], ) -> Arc { - self.try_run2(plan, optimizers_to_run).unwrap() + self.try_run(plan, optimizers_to_run).unwrap() } } @@ -692,7 +617,7 @@ fn multi_hash_joins() -> Result<()> { ); let test_config = TestConfig::default(); - let plan_distrub = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrub = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); match join_type { // Should include 3 RepartitionExecs @@ -734,7 +659,7 @@ fn multi_hash_joins() -> Result<()> { }; - let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrub, plan_sort); } JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => {} @@ -761,7 +686,7 @@ fn multi_hash_joins() -> Result<()> { hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); let test_config = TestConfig::default(); - let plan_distrub = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrub = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); match join_type { // Should include 3 RepartitionExecs @@ -809,7 +734,7 @@ fn multi_hash_joins() -> Result<()> { }; - let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrub, plan_sort); } JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} @@ -856,7 +781,7 @@ fn multi_joins_after_alias() -> Result<()> { // Output partition need to respect the Alias and should not introduce additional RepartitionExec let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -874,7 +799,7 @@ fn multi_joins_after_alias() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); // Join on (a2 == c) @@ -887,7 +812,7 @@ fn multi_joins_after_alias() -> Result<()> { // Output partition need to respect the Alias and should not introduce additional RepartitionExec let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -905,7 +830,7 @@ fn multi_joins_after_alias() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -943,7 +868,7 @@ fn multi_joins_after_multi_alias() -> Result<()> { // The Column 'a' has different meaning now after the two Projections // The original Output partition can not satisfy the Join requirements and need to add an additional RepartitionExec let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -963,7 +888,7 @@ fn multi_joins_after_multi_alias() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -991,7 +916,7 @@ fn join_after_agg_alias() -> Result<()> { // Only two RepartitionExecs added let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -1008,7 +933,7 @@ fn join_after_agg_alias() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -1048,7 +973,7 @@ fn hash_join_key_ordering() -> Result<()> { // Only two RepartitionExecs added let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -1066,7 +991,7 @@ fn hash_join_key_ordering() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -1172,7 +1097,7 @@ fn multi_hash_join_key_ordering() -> Result<()> { // The bottom joins' join key ordering is adjusted based on the top join. And the top join should not introduce additional RepartitionExec let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(filter_top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(filter_top_join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -1197,7 +1122,7 @@ fn multi_hash_join_key_ordering() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(filter_top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(filter_top_join, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -1525,7 +1450,7 @@ fn multi_smj_joins() -> Result<()> { let mut settings = Settings::clone_current(); settings.add_filter(&format!("join_type={}", join_type), "join_type=..."); settings.bind(|| { - let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); match join_type { // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs @@ -1582,7 +1507,7 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] } } - let plan_sort = test_config.run2(top_join.clone(), &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join.clone(), &SORT_DISTRIB_DISTRIB); match join_type { // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs @@ -1654,7 +1579,7 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] let top_join = sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); - let plan_distrib = test_config.run2(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); match join_type { // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs @@ -1707,7 +1632,7 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] _ => unreachable!() } - let plan_sort = test_config.run2(top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); match join_type { // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs @@ -1823,7 +1748,7 @@ fn smj_join_key_ordering() -> Result<()> { // Test: run EnforceDistribution, then EnforceSort. // Only two RepartitionExecs added - let plan_distrib = test_config.run2(join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -1847,7 +1772,7 @@ SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let plan_sort = test_config.run2(join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(join, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -1904,7 +1829,7 @@ fn merge_does_not_need_sort() -> Result<()> { // The optimizer should not add an additional SortExec as the // data is already sorted let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(exec.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(exec.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -1920,7 +1845,7 @@ SortPreservingMergeExec: [a@0 ASC] // (according to flag: PREFER_EXISTING_SORT) // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with // SortExec at the top. - let plan_sort = test_config.run2(exec, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(exec, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -1956,7 +1881,7 @@ fn union_to_interleave() -> Result<()> { // Only two RepartitionExecs added, no final RepartitionExec required let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -1975,7 +1900,7 @@ fn union_to_interleave() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2005,7 +1930,7 @@ fn union_not_to_interleave() -> Result<()> { // TestConfig: Prefer existing union. let test_config = TestConfig::default().with_prefer_existing_union(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2025,7 +1950,7 @@ fn union_not_to_interleave() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2037,7 +1962,7 @@ fn added_repartition_to_single_partition() -> Result<()> { let plan = aggregate_exec_with_alias(parquet_exec(), alias); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2048,7 +1973,7 @@ fn added_repartition_to_single_partition() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2060,7 +1985,7 @@ fn repartition_deepest_node() -> Result<()> { let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2072,7 +1997,7 @@ fn repartition_deepest_node() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2083,7 +2008,7 @@ fn repartition_unsorted_limit() -> Result<()> { let plan = limit_exec(filter_exec(parquet_exec())); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2095,7 +2020,7 @@ fn repartition_unsorted_limit() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2112,7 +2037,7 @@ fn repartition_sorted_limit() -> Result<()> { let plan = limit_exec(sort_exec(sort_key, parquet_exec())); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2122,7 +2047,7 @@ GlobalLimitExec: skip=0, fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2142,7 +2067,7 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2153,7 +2078,7 @@ SortRequiredExec: [c@2 ASC] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2168,7 +2093,7 @@ fn repartition_ignores_limit() -> Result<()> { ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2186,7 +2111,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2197,7 +2122,7 @@ fn repartition_ignores_union() -> Result<()> { let plan = union_exec(vec![parquet_exec(); 5]); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2209,7 +2134,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2227,7 +2152,7 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2235,7 +2160,7 @@ SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2256,7 +2181,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2266,7 +2191,7 @@ SortPreservingMergeExec: [c@2 ASC] ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -2295,7 +2220,7 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let plan = sort_preserving_merge_exec(sort_key, input); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2307,7 +2232,7 @@ SortPreservingMergeExec: [c@2 ASC] ); // test: result IS DIFFERENT, if EnforceSorting is run first: - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -2341,7 +2266,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { // TestConfig: Prefer existing sort. let test_config = TestConfig::default().with_prefer_existing_sort(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2351,7 +2276,7 @@ SortRequiredExec: [d@3 ASC] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2382,7 +2307,7 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { let plan = union_exec(vec![input1, input2]); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2394,7 +2319,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2421,7 +2346,7 @@ fn repartition_transitively_with_projection() -> Result<()> { let plan = sort_preserving_merge_exec(sort_key, proj); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2434,7 +2359,7 @@ SortPreservingMergeExec: [sum@0 ASC] ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -2472,7 +2397,7 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2481,7 +2406,7 @@ SortRequiredExec: [c@2 ASC] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2509,7 +2434,7 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2518,7 +2443,7 @@ SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -2535,7 +2460,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { let plan = sort_exec(sort_key, filter_exec(parquet_exec())); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2548,7 +2473,7 @@ SortPreservingMergeExec: [a@0 ASC] ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -2585,7 +2510,7 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -2599,7 +2524,7 @@ SortPreservingMergeExec: [a@0 ASC] ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -2627,7 +2552,7 @@ fn parallelization_single_partition() -> Result<()> { // Test: with parquet let plan_parquet_distrib = - test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @r" @@ -2637,11 +2562,11 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_csv_distrib, @r" @@ -2651,7 +2576,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -2679,7 +2604,7 @@ fn parallelization_multiple_files() -> Result<()> { let test_config_concurrency_3 = test_config.clone().with_query_execution_partitions(3); let plan_3_distrib = - test_config_concurrency_3.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + test_config_concurrency_3.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_3_distrib, @r" @@ -2688,12 +2613,12 @@ SortRequiredExec: [a@0 ASC] DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " ); - let plan_3_sort = test_config_concurrency_3.run2(plan.clone(), &SORT_DISTRIB_DISTRIB); + let plan_3_sort = test_config_concurrency_3.run(plan.clone(), &SORT_DISTRIB_DISTRIB); assert_plan!(plan_3_distrib, plan_3_sort); let test_config_concurrency_8 = test_config.with_query_execution_partitions(8); let plan_8_distrib = - test_config_concurrency_8.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + test_config_concurrency_8.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_8_distrib, @r" @@ -2702,7 +2627,7 @@ SortRequiredExec: [a@0 ASC] DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " ); - let plan_8_sort = test_config_concurrency_8.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_8_sort = test_config_concurrency_8.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_8_distrib, plan_8_sort); Ok(()) @@ -2739,7 +2664,7 @@ fn parallelization_compressed_csv() -> Result<()> { .with_query_execution_partitions(2) .with_prefer_repartition_file_scans(10); - let plan_distrib = test_config.run2(plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); if compression_type.is_compressed() { // Compressed files cannot be partitioned assert_plan!( @@ -2765,7 +2690,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] ); } - let plan_sort = test_config.run2(plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); } } @@ -2783,7 +2708,7 @@ fn parallelization_two_partitions() -> Result<()> { .with_prefer_repartition_file_scans(10); // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @r" @@ -2793,11 +2718,11 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_csv_distrib, @r" @@ -2807,7 +2732,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -2824,7 +2749,7 @@ fn parallelization_two_partitions_into_four() -> Result<()> { .with_prefer_repartition_file_scans(10); // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions assert_plan!( plan_parquet_distrib, @@ -2835,11 +2760,11 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions assert_plan!( plan_csv_distrib, @@ -2850,7 +2775,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -2870,7 +2795,7 @@ fn parallelization_sorted_limit() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // data is sorted so can't repartition here // Doesn't parallelize for SortExec without preserve_partitioning assert_plan!( @@ -2882,11 +2807,11 @@ GlobalLimitExec: skip=0, fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // data is sorted so can't repartition here // Doesn't parallelize for SortExec without preserve_partitioning assert_plan!( @@ -2898,7 +2823,7 @@ GlobalLimitExec: skip=0, fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -2919,7 +2844,7 @@ fn parallelization_limit_with_filter() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. // SortExec doesn't benefit from input partitioning @@ -2935,11 +2860,11 @@ GlobalLimitExec: skip=0, fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. // SortExec doesn't benefit from input partitioning @@ -2955,7 +2880,7 @@ GlobalLimitExec: skip=0, fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -2975,7 +2900,7 @@ fn parallelization_ignores_limit() -> Result<()> { // Test: with parquet let plan_parquet_distrib = - test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @r" @@ -2993,11 +2918,11 @@ fn parallelization_ignores_limit() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_csv_distrib, @r" @@ -3015,7 +2940,7 @@ fn parallelization_ignores_limit() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -3029,7 +2954,7 @@ fn parallelization_union_inputs() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Union doesn't benefit from input partitioning - no parallelism assert_plan!( plan_parquet_distrib, @@ -3042,11 +2967,11 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Union doesn't benefit from input partitioning - no parallelism assert_plan!( plan_csv_distrib, @@ -3059,7 +2984,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -3088,21 +3013,21 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // Test: with parquet let plan_parquet_distrib = - test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet" ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_csv_distrib, @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false" ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -3134,7 +3059,7 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { // Test: with parquet let plan_parquet_distrib = - test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @r" @@ -3144,7 +3069,7 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_parquet_sort, @r" @@ -3157,7 +3082,7 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { ); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_csv_distrib, @r" @@ -3167,7 +3092,7 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv.clone(), &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv.clone(), &SORT_DISTRIB_DISTRIB); assert_plan!( plan_csv_sort, @r" @@ -3206,7 +3131,7 @@ fn parallelization_does_not_benefit() -> Result<()> { // Test: with parquet let plan_parquet_distrib = - test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_parquet_distrib, @r" @@ -3214,11 +3139,11 @@ fn parallelization_does_not_benefit() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv - let plan_csv_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_csv_distrib, @r" @@ -3226,7 +3151,7 @@ fn parallelization_does_not_benefit() -> Result<()> { DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " ); - let plan_csv_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) @@ -3269,7 +3194,7 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> ); let test_config = TestConfig::default(); - let plan_parquet_distrib = test_config.run2(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Expected Outcome: // data should not be repartitioned / resorted assert_plan!( @@ -3279,7 +3204,7 @@ ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_parquet_sort = test_config.run2(plan_parquet, &SORT_DISTRIB_DISTRIB); + let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); Ok(()) @@ -3319,7 +3244,7 @@ SortPreservingMergeExec: [c2@1 ASC] ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3327,7 +3252,7 @@ ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " ); - let plan_sort = test_config.run2(plan_csv, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3350,7 +3275,7 @@ RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3359,7 +3284,7 @@ FilterExec: c@2 = 0 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3380,7 +3305,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { // TestConfig: Prefer existing sort. let test_config = TestConfig::default().with_prefer_existing_sort(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3390,7 +3315,7 @@ CoalescePartitionsExec DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3411,7 +3336,7 @@ fn preserve_ordering_through_repartition() -> Result<()> { // TestConfig: Prefer existing sort. let test_config = TestConfig::default().with_prefer_existing_sort(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3421,7 +3346,7 @@ SortPreservingMergeExec: [d@3 ASC] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3440,7 +3365,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3453,7 +3378,7 @@ SortPreservingMergeExec: [a@0 ASC] ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -3480,7 +3405,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3490,7 +3415,7 @@ CoalescePartitionsExec DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3515,7 +3440,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3528,7 +3453,7 @@ SortPreservingMergeExec: [a@0 ASC] ); // Test: result IS DIFFERENT, if EnforceSorting is run first: - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!( plan_sort, @r" @@ -3556,7 +3481,7 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let physical_plan = filter_exec(input); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3565,7 +3490,7 @@ FilterExec: c@2 = 0 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3663,7 +3588,7 @@ fn do_not_add_unnecessary_hash() -> Result<()> { // Make sure target partition number is 1. In this case hash repartition is unnecessary. let test_config = TestConfig::default().with_query_execution_partitions(1); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3672,7 +3597,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3695,7 +3620,7 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { // Make sure target partition number is larger than 2 (e.g partition number at the source). let test_config = TestConfig::default().with_query_execution_partitions(4); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3708,7 +3633,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3727,14 +3652,14 @@ CoalescePartitionsExec ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3758,7 +3683,7 @@ FilterExec: c@2 = 0 ); let test_config = TestConfig::default(); - let plan_distrib = test_config.run2(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( plan_distrib, @r" @@ -3768,7 +3693,7 @@ FilterExec: c@2 = 0 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " ); - let plan_sort = test_config.run2(physical_plan, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); Ok(()) @@ -3798,7 +3723,7 @@ SortExec: expr=[c@2 ASC], preserve_partitioning=[false] ); // what the enforce distribution run does. - let plan_distribution = test_config.run2(physical_plan.clone(), &[Run::Distribution]); + let plan_distribution = test_config.run(physical_plan.clone(), &[Run::Distribution]); assert_plan!( plan_distribution, @r" @@ -3809,7 +3734,7 @@ SortExec: expr=[c@2 ASC], preserve_partitioning=[false] ); // what the sort parallelization (in enforce sorting), does after the enforce distribution changes - let plan_both = test_config.run2(physical_plan, &[Run::Distribution, Run::Sorting]); + let plan_both = test_config.run(physical_plan, &[Run::Distribution, Run::Sorting]); assert_plan!( plan_both, @r" From f6366b8f47f5d1ffa8e6771d364fe162b765f7c3 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 21:38:28 +0100 Subject: [PATCH 36/41] Migrate remaining assert_plan_txt! tests and remove macro MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Migrated the last two tests using assert_plan_txt! macro to insta snapshots and removed the macro entirely. Tests migrated: - do_not_put_sort_when_input_is_invalid - put_sort_when_input_is_valid Key changes: - Converted assert_plan_txt! calls to assert_plan! with inline snapshots - Preserved all comments explaining expected behavior - Removed assert_plan_txt! macro definition - Removed unused get_plan_string import The assert_plan_txt! macro is now completely eliminated from the codebase. 🤖 Generated with [Claude Code](https://claude.com/claude-code) Co-Authored-By: Claude --- .../enforce_distribution.rs | 83 ++++++++----------- 1 file changed, 34 insertions(+), 49 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 3319d56e0d553..19a38c069ecb8 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -66,7 +66,7 @@ use datafusion_physical_plan::projection::{ProjectionExec, ProjectionExpr}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{ - displayable, get_plan_string, DisplayAs, DisplayFormatType, ExecutionPlanProperties, + displayable, DisplayAs, DisplayFormatType, ExecutionPlanProperties, PlanProperties, Statistics, }; use insta::Settings; @@ -537,19 +537,6 @@ impl TestConfig { } } -macro_rules! assert_plan_txt { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); - // Now format correctly - let actual_lines = get_plan_string(&$PLAN); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; -} #[test] fn multi_hash_joins() -> Result<()> { @@ -3506,30 +3493,29 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { .into(); let input = parquet_exec(); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); - let expected = &[ - // Ordering requirement of sort required exec is NOT satisfied - // by existing ordering at the source. - "SortRequiredExec: [a@0 ASC]", - " FilterExec: c@2 = 0", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; - assert_plan_txt!(expected, physical_plan); - - let expected = &[ - "SortRequiredExec: [a@0 ASC]", - // Since at the start of the rule ordering requirement is not satisfied - // EnforceDistribution rule doesn't satisfy this requirement either. - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ]; + // Ordering requirement of sort required exec is NOT satisfied + // by existing ordering at the source. + assert_plan!(physical_plan, @r" +SortRequiredExec: [a@0 ASC] + FilterExec: c@2 = 0 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; config.optimizer.enable_round_robin_repartition = true; config.optimizer.prefer_existing_sort = false; let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; - assert_plan_txt!(expected, dist_plan); + // Since at the start of the rule ordering requirement is not satisfied + // EnforceDistribution rule doesn't satisfy this requirement either. + assert_plan!(dist_plan, @r" +SortRequiredExec: [a@0 ASC] + FilterExec: c@2 = 0 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +" + ); Ok(()) } @@ -3545,29 +3531,28 @@ fn put_sort_when_input_is_valid() -> Result<()> { let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); - let expected = &[ - // Ordering requirement of sort required exec is satisfied - // by existing ordering at the source. - "SortRequiredExec: [a@0 ASC]", - " FilterExec: c@2 = 0", - " DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; - assert_plan_txt!(expected, physical_plan); - - let expected = &[ - // Since at the start of the rule ordering requirement is satisfied - // EnforceDistribution rule satisfy this requirement also. - "SortRequiredExec: [a@0 ASC]", - " FilterExec: c@2 = 0", - " DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", - ]; + // Ordering requirement of sort required exec is satisfied + // by existing ordering at the source. + assert_plan!(physical_plan, @r" +SortRequiredExec: [a@0 ASC] + FilterExec: c@2 = 0 + DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet +" + ); let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; config.optimizer.enable_round_robin_repartition = true; config.optimizer.prefer_existing_sort = false; let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; - assert_plan_txt!(expected, dist_plan); + // Since at the start of the rule ordering requirement is satisfied + // EnforceDistribution rule satisfy this requirement also. + assert_plan!(dist_plan, @r" +SortRequiredExec: [a@0 ASC] + FilterExec: c@2 = 0 + DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet +" + ); Ok(()) } From c02b9edca5ecb3f051d0277a218628797101bac5 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 21:39:29 +0100 Subject: [PATCH 37/41] fmt --- .../enforce_distribution.rs | 735 +++++++++--------- 1 file changed, 373 insertions(+), 362 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 19a38c069ecb8..c0e693ceccb29 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -66,20 +66,27 @@ use datafusion_physical_plan::projection::{ProjectionExec, ProjectionExpr}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{ - displayable, DisplayAs, DisplayFormatType, ExecutionPlanProperties, - PlanProperties, Statistics, + displayable, DisplayAs, DisplayFormatType, ExecutionPlanProperties, PlanProperties, + Statistics, }; use insta::Settings; /// Helper function to replace only the first occurrence of a regex pattern in a plan /// Returns (captured_group_1, modified_string) -fn hide_first(plan: &dyn ExecutionPlan, regex: &str, replacement: &str) -> (String, String) { +fn hide_first( + plan: &dyn ExecutionPlan, + regex: &str, + replacement: &str, +) -> (String, String) { let plan_str = displayable(plan).indent(true).to_string(); let pattern = regex::Regex::new(regex).unwrap(); if let Some(captures) = pattern.captures(&plan_str) { let full_match = captures.get(0).unwrap(); - let captured_value = captures.get(1).map(|m| m.as_str().to_string()).unwrap_or_default(); + let captured_value = captures + .get(1) + .map(|m| m.as_str().to_string()) + .unwrap_or_default(); let pos = full_match.start(); let end_pos = full_match.end(); let mut result = String::with_capacity(plan_str.len()); @@ -462,7 +469,6 @@ impl TestConfig { self } - /// Perform a series of runs using the current [`TestConfig`], /// assert the expected plan result, /// and return the result plan (for potential subsequent runs). @@ -537,7 +543,6 @@ impl TestConfig { } } - #[test] fn multi_hash_joins() -> Result<()> { let left = parquet_exec(); @@ -1215,21 +1220,21 @@ fn reorder_join_keys_to_left_input() -> Result<()> { ]; insta::allow_duplicates! { - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); - let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); + let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); - // The top joins' join key ordering is adjusted based on the children inputs. - let (captured_join_type, modified_plan) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); - assert_eq!(captured_join_type, join_type.to_string()); + // The top joins' join key ordering is adjusted based on the children inputs. + let (captured_join_type, modified_plan) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); + assert_eq!(captured_join_type, join_type.to_string()); - insta::assert_snapshot!(modified_plan, @r" + insta::assert_snapshot!(modified_plan, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)] ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)] @@ -1249,9 +1254,9 @@ HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } } - } Ok(()) } @@ -1351,19 +1356,19 @@ fn reorder_join_keys_to_right_input() -> Result<()> { ]; insta::allow_duplicates! { - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); - let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); + let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); - // The top joins' join key ordering is adjusted based on the children inputs. - let (_, plan_str) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); - insta::assert_snapshot!(plan_str, @r" + // The top joins' join key ordering is adjusted based on the children inputs. + let (_, plan_str) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); + insta::assert_snapshot!(plan_str, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)] ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)] @@ -1383,9 +1388,9 @@ HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (AA@ ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } } - } Ok(()) } @@ -1422,27 +1427,27 @@ fn multi_smj_joins() -> Result<()> { )]; insta::allow_duplicates! { - for join_type in join_types { - let join = - sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - - // Top join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); - - let mut settings = Settings::clone_current(); - settings.add_filter(&format!("join_type={}", join_type), "join_type=..."); - settings.bind(|| { - let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + for join_type in join_types { + let join = + sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - assert_plan!(plan_distrib, @r" + // Top join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); + + let mut settings = Settings::clone_current(); + settings.add_filter(&format!("join_type={}", join_type), "join_type=..."); + settings.bind(|| { + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -1459,19 +1464,19 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); - } - // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test - // cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => { - assert_plan!(plan_distrib, @r" + ); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test + // cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 @@ -1490,17 +1495,17 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } } - } - let plan_sort = test_config.run(top_join.clone(), &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join.clone(), &SORT_DISTRIB_DISTRIB); - match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC @@ -1517,20 +1522,20 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); - } - // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition and Roundrobin repartition after - // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + ); + } + // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition and Roundrobin repartition after + // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1551,28 +1556,28 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } } - } - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); - - let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); - - match join_type { - // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs - JoinType::Inner | JoinType::Right => { - // TODO(wiedld): show different test result if enforce sorting first. - assert_plan!(plan_distrib, @r" + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); + + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { + // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -1589,12 +1594,12 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); - } - // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs - JoinType::Left | JoinType::Full => { - // TODO(wiedld): show different test result if enforce sorting first. - assert_plan!(plan_distrib, @r" + ); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortExec: expr=[b1@6 ASC], preserve_partitioning=[true] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 @@ -1613,19 +1618,19 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } + // this match arm cannot be reached + _ => unreachable!() } - // this match arm cannot be reached - _ => unreachable!() - } - let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); - match join_type { - // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs - JoinType::Inner | JoinType::Right => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + match join_type { + // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC @@ -1642,12 +1647,12 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); - } - // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs - JoinType::Left | JoinType::Full => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + ); + } + // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1668,17 +1673,17 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } + // this match arm cannot be reached + _ => unreachable!() } - // this match arm cannot be reached - _ => unreachable!() } + _ => {} } - _ => {} - } - }); + }); + } } - } Ok(()) } @@ -1737,8 +1742,8 @@ fn smj_join_key_ordering() -> Result<()> { // Only two RepartitionExecs added let plan_distrib = test_config.run(join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a1@0 as a3, b1@1 as b3] @@ -1756,13 +1761,13 @@ SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(join, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1786,7 +1791,7 @@ SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -1818,13 +1823,13 @@ fn merge_does_not_need_sort() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(exec.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] CoalesceBatchesExec: target_batch_size=4096 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: // @@ -1834,14 +1839,14 @@ SortPreservingMergeExec: [a@0 ASC] // SortExec at the top. let plan_sort = test_config.run(exec, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec CoalesceBatchesExec: target_batch_size=4096 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -2026,14 +2031,14 @@ fn repartition_sorted_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2056,15 +2061,15 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [c@2 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2082,8 +2087,8 @@ fn repartition_ignores_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] @@ -2097,7 +2102,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] LocalLimitExec: fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2111,8 +2116,8 @@ fn repartition_ignores_union() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet @@ -2120,7 +2125,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2141,12 +2146,12 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2170,23 +2175,23 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [c@2 ASC] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -2209,27 +2214,27 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [c@2 ASC] UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -2255,14 +2260,14 @@ fn repartition_does_not_destroy_sort() -> Result<()> { let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [d@3 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2296,8 +2301,8 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" UnionExec SortRequiredExec: [c@2 ASC] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet @@ -2305,7 +2310,7 @@ UnionExec RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2335,28 +2340,28 @@ fn repartition_transitively_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [sum@0 ASC] SortExec: expr=[sum@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a@0 + b@1 as sum] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[sum@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec ProjectionExec: expr=[a@0 + b@1 as sum] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2386,13 +2391,13 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [c@2 ASC] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2423,13 +2428,13 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2449,28 +2454,28 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2499,8 +2504,8 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] @@ -2508,13 +2513,13 @@ SortPreservingMergeExec: [a@0 ASC] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] @@ -2522,7 +2527,7 @@ SortExec: expr=[a@0 ASC], preserve_partitioning=[false] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2541,28 +2546,28 @@ fn parallelization_single_partition() -> Result<()> { let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2593,13 +2598,13 @@ fn parallelization_multiple_files() -> Result<()> { let plan_3_distrib = test_config_concurrency_3.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_3_distrib, - @r" + plan_3_distrib, + @r" SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); let plan_3_sort = test_config_concurrency_3.run(plan.clone(), &SORT_DISTRIB_DISTRIB); assert_plan!(plan_3_distrib, plan_3_sort); @@ -2607,13 +2612,13 @@ SortRequiredExec: [a@0 ASC] let plan_8_distrib = test_config_concurrency_8.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_8_distrib, - @r" + plan_8_distrib, + @r" SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); let plan_8_sort = test_config_concurrency_8.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_8_distrib, plan_8_sort); @@ -2633,54 +2638,54 @@ fn parallelization_compressed_csv() -> Result<()> { ]; insta::allow_duplicates! { - for compression_type in compression_types { - let plan = aggregate_exec_with_alias( - DataSourceExec::from_data_source( - FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_file_compression_type(compression_type) - .build(), - ), - vec![("a".to_string(), "a".to_string())], - ); - let test_config = TestConfig::default() - .with_query_execution_partitions(2) - .with_prefer_repartition_file_scans(10); - - let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - if compression_type.is_compressed() { - // Compressed files cannot be partitioned - assert_plan!( - plan_distrib, - @r" + for compression_type in compression_types { + let plan = aggregate_exec_with_alias( + DataSourceExec::from_data_source( + FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_file_compression_type(compression_type) + .build(), + ), + vec![("a".to_string(), "a".to_string())], + ); + let test_config = TestConfig::default() + .with_query_execution_partitions(2) + .with_prefer_repartition_file_scans(10); + + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); + if compression_type.is_compressed() { + // Compressed files cannot be partitioned + assert_plan!( + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); - } else { - // Uncompressed files can be partitioned - assert_plan!( - plan_distrib, - @r" + ); + } else { + // Uncompressed files can be partitioned + assert_plan!( + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); - } + ); + } - let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); - assert_plan!(plan_distrib, plan_sort); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); + } } - } Ok(()) } @@ -2695,30 +2700,31 @@ fn parallelization_two_partitions() -> Result<()> { .with_prefer_repartition_file_scans(10); // Test: with parquet - let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2736,17 +2742,18 @@ fn parallelization_two_partitions_into_four() -> Result<()> { .with_prefer_repartition_file_scans(10); // Test: with parquet - let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2754,14 +2761,14 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2782,18 +2789,19 @@ fn parallelization_sorted_limit() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // data is sorted so can't repartition here // Doesn't parallelize for SortExec without preserve_partitioning assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2802,14 +2810,14 @@ GlobalLimitExec: skip=0, fetch=100 // data is sorted so can't repartition here // Doesn't parallelize for SortExec without preserve_partitioning assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2831,13 +2839,14 @@ fn parallelization_limit_with_filter() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. // SortExec doesn't benefit from input partitioning assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec LocalLimitExec: fetch=100 @@ -2846,7 +2855,7 @@ GlobalLimitExec: skip=0, fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2856,8 +2865,8 @@ GlobalLimitExec: skip=0, fetch=100 // ordering is not used in subsequent stages anyway. // SortExec doesn't benefit from input partitioning assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec LocalLimitExec: fetch=100 @@ -2866,7 +2875,7 @@ GlobalLimitExec: skip=0, fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2941,11 +2950,12 @@ fn parallelization_union_inputs() -> Result<()> { let test_config = TestConfig::default(); // Test: with parquet - let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Union doesn't benefit from input partitioning - no parallelism assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet @@ -2953,7 +2963,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2961,8 +2971,8 @@ UnionExec let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Union doesn't benefit from input partitioning - no parallelism assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false @@ -2970,7 +2980,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -3181,16 +3191,17 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> ); let test_config = TestConfig::default(); - let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); + let plan_parquet_distrib = + test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Expected Outcome: // data should not be repartitioned / resorted assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -3222,23 +3233,23 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { .into(); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); assert_plan!( - plan_csv, - @r" + plan_csv, + @r" SortPreservingMergeExec: [c2@1 ASC] ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " - ); + ); let plan_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3251,26 +3262,26 @@ fn remove_redundant_roundrobins() -> Result<()> { let repartition = repartition_exec(repartition_exec(input)); let physical_plan = repartition_exec(filter_exec(repartition)); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3294,14 +3305,14 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3325,14 +3336,14 @@ fn preserve_ordering_through_repartition() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [d@3 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3354,28 +3365,28 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -3394,14 +3405,14 @@ fn no_need_for_sort_after_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3429,21 +3440,21 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -3451,7 +3462,7 @@ SortExec: expr=[a@0 ASC], preserve_partitioning=[false] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -3470,13 +3481,13 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3500,7 +3511,7 @@ SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; @@ -3515,7 +3526,7 @@ SortRequiredExec: [a@0 ASC] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -3538,7 +3549,7 @@ SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; @@ -3552,7 +3563,7 @@ SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -3575,13 +3586,13 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3607,8 +3618,8 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] @@ -3617,7 +3628,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3628,22 +3639,22 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] fn optimize_away_unnecessary_repartition() -> Result<()> { let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" CoalescePartitionsExec RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3656,8 +3667,8 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { filter_exec(repartition_exec(parquet_exec())), ))); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 CoalescePartitionsExec @@ -3665,19 +3676,19 @@ FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3700,34 +3711,34 @@ async fn test_distribute_sort_parquet() -> Result<()> { // prior to optimization, this is the starting plan assert_plan!( - physical_plan, - @r" + physical_plan, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // what the enforce distribution run does. let plan_distribution = test_config.run(physical_plan.clone(), &[Run::Distribution]); assert_plan!( - plan_distribution, - @r" + plan_distribution, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // what the sort parallelization (in enforce sorting), does after the enforce distribution changes let plan_both = test_config.run(physical_plan, &[Run::Distribution, Run::Sorting]); assert_plan!( - plan_both, - @r" + plan_both, + @r" SortPreservingMergeExec: [c@2 ASC] SortExec: expr=[c@2 ASC], preserve_partitioning=[true] DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -3753,13 +3764,13 @@ async fn test_distribute_sort_memtable() -> Result<()> { // this is the final, optimized plan assert_plan!( - physical_plan, - @r" + physical_plan, + @r" SortPreservingMergeExec: [id@0 ASC NULLS LAST] SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] DataSourceExec: partitions=3, partition_sizes=[34, 33, 33] " - ); + ); Ok(()) } From 18d9254ffca18f2f87162e4b209efd0b3eb55bd4 Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 20 Oct 2025 21:43:27 +0100 Subject: [PATCH 38/41] clippy --- .../core/tests/physical_optimizer/enforce_distribution.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index c0e693ceccb29..9726ef88c6f19 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1440,7 +1440,7 @@ fn multi_smj_joins() -> Result<()> { sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); let mut settings = Settings::clone_current(); - settings.add_filter(&format!("join_type={}", join_type), "join_type=..."); + settings.add_filter(&format!("join_type={join_type}"), "join_type=..."); settings.bind(|| { let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); From dbae19d43bdc1faaae62cbd6fbe39a5e43563596 Mon Sep 17 00:00:00 2001 From: blaginin Date: Tue, 21 Oct 2025 09:56:36 +0100 Subject: [PATCH 39/41] typo and fmt --- .../enforce_distribution.rs | 718 +++++++++--------- 1 file changed, 359 insertions(+), 359 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 9726ef88c6f19..ec3a2f3768815 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -609,13 +609,13 @@ fn multi_hash_joins() -> Result<()> { ); let test_config = TestConfig::default(); - let plan_distrub = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); match join_type { // Should include 3 RepartitionExecs JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { - assert_plan!(plan_distrub, @r" + assert_plan!(plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, c@2)] HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 @@ -632,7 +632,7 @@ fn multi_hash_joins() -> Result<()> { }, // Should include 4 RepartitionExecs _ => { - assert_plan!(plan_distrub, @r" + assert_plan!(plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] @@ -652,7 +652,7 @@ fn multi_hash_joins() -> Result<()> { let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); - assert_plan!(plan_distrub, plan_sort); + assert_plan!(plan_distrib, plan_sort); } JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => {} } @@ -678,7 +678,7 @@ fn multi_hash_joins() -> Result<()> { hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); let test_config = TestConfig::default(); - let plan_distrub = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); match join_type { // Should include 3 RepartitionExecs @@ -687,7 +687,7 @@ fn multi_hash_joins() -> Result<()> { }, // Should include 3 RepartitionExecs but have a different "on" JoinType::RightSemi | JoinType::RightAnti => { - assert_plan!(plan_distrub, @r" + assert_plan!(plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(b1@1, c@2)] HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 @@ -706,7 +706,7 @@ fn multi_hash_joins() -> Result<()> { // Should include 4 RepartitionExecs _ => { - assert_plan!(plan_distrub, @r" + assert_plan!(plan_distrib, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 HashJoinExec: mode=Partitioned, join_type=..., on=[(a@0, b1@1)] @@ -727,7 +727,7 @@ fn multi_hash_joins() -> Result<()> { let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); - assert_plan!(plan_distrub, plan_sort); + assert_plan!(plan_distrib, plan_sort); } JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} } @@ -1220,21 +1220,21 @@ fn reorder_join_keys_to_left_input() -> Result<()> { ]; insta::allow_duplicates! { - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); - let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); + let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); - // The top joins' join key ordering is adjusted based on the children inputs. - let (captured_join_type, modified_plan) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); - assert_eq!(captured_join_type, join_type.to_string()); + // The top joins' join key ordering is adjusted based on the children inputs. + let (captured_join_type, modified_plan) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); + assert_eq!(captured_join_type, join_type.to_string()); - insta::assert_snapshot!(modified_plan, @r" + insta::assert_snapshot!(modified_plan, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)] ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)] @@ -1254,9 +1254,9 @@ HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } } - } Ok(()) } @@ -1356,19 +1356,19 @@ fn reorder_join_keys_to_right_input() -> Result<()> { ]; insta::allow_duplicates! { - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); - let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); + let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); - // The top joins' join key ordering is adjusted based on the children inputs. - let (_, plan_str) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); - insta::assert_snapshot!(plan_str, @r" + // The top joins' join key ordering is adjusted based on the children inputs. + let (_, plan_str) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); + insta::assert_snapshot!(plan_str, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)] ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)] @@ -1388,9 +1388,9 @@ HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (AA@ ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } } - } Ok(()) } @@ -1427,27 +1427,27 @@ fn multi_smj_joins() -> Result<()> { )]; insta::allow_duplicates! { - for join_type in join_types { - let join = - sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - - // Top join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); - - let mut settings = Settings::clone_current(); - settings.add_filter(&format!("join_type={join_type}"), "join_type=..."); - settings.bind(|| { - let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); - - match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - assert_plan!(plan_distrib, @r" + for join_type in join_types { + let join = + sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); + + // Top join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); + + let mut settings = Settings::clone_current(); + settings.add_filter(&format!("join_type={join_type}"), "join_type=..."); + settings.bind(|| { + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -1464,19 +1464,19 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); - } - // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test - // cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => { - assert_plan!(plan_distrib, @r" + ); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test + // cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 @@ -1495,17 +1495,17 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } } - } - let plan_sort = test_config.run(top_join.clone(), &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join.clone(), &SORT_DISTRIB_DISTRIB); - match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC @@ -1522,20 +1522,20 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); - } - // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition and Roundrobin repartition after - // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + ); + } + // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition and Roundrobin repartition after + // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1556,28 +1556,28 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } } - } - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); - - let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); - - match join_type { - // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs - JoinType::Inner | JoinType::Right => { - // TODO(wiedld): show different test result if enforce sorting first. - assert_plan!(plan_distrib, @r" + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); + + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { + // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -1594,12 +1594,12 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); - } - // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs - JoinType::Left | JoinType::Full => { - // TODO(wiedld): show different test result if enforce sorting first. - assert_plan!(plan_distrib, @r" + ); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortExec: expr=[b1@6 ASC], preserve_partitioning=[true] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 @@ -1618,19 +1618,19 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } + // this match arm cannot be reached + _ => unreachable!() } - // this match arm cannot be reached - _ => unreachable!() - } - let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); - match join_type { - // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs - JoinType::Inner | JoinType::Right => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + match join_type { + // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC @@ -1647,12 +1647,12 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); - } - // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs - JoinType::Left | JoinType::Full => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + ); + } + // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1673,17 +1673,17 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); + } + // this match arm cannot be reached + _ => unreachable!() } - // this match arm cannot be reached - _ => unreachable!() } + _ => {} } - _ => {} - } - }); + }); + } } - } Ok(()) } @@ -1742,8 +1742,8 @@ fn smj_join_key_ordering() -> Result<()> { // Only two RepartitionExecs added let plan_distrib = test_config.run(join.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a1@0 as a3, b1@1 as b3] @@ -1761,13 +1761,13 @@ SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(join, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1791,7 +1791,7 @@ SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -1823,13 +1823,13 @@ fn merge_does_not_need_sort() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(exec.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] CoalesceBatchesExec: target_batch_size=4096 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: // @@ -1839,14 +1839,14 @@ SortPreservingMergeExec: [a@0 ASC] // SortExec at the top. let plan_sort = test_config.run(exec, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec CoalesceBatchesExec: target_batch_size=4096 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -2031,14 +2031,14 @@ fn repartition_sorted_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2061,15 +2061,15 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [c@2 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2087,8 +2087,8 @@ fn repartition_ignores_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] @@ -2102,7 +2102,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] LocalLimitExec: fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2116,8 +2116,8 @@ fn repartition_ignores_union() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet @@ -2125,7 +2125,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2146,12 +2146,12 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2175,23 +2175,23 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [c@2 ASC] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -2214,27 +2214,27 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [c@2 ASC] UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -2260,14 +2260,14 @@ fn repartition_does_not_destroy_sort() -> Result<()> { let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [d@3 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2301,8 +2301,8 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" UnionExec SortRequiredExec: [c@2 ASC] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet @@ -2310,7 +2310,7 @@ UnionExec RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2340,28 +2340,28 @@ fn repartition_transitively_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [sum@0 ASC] SortExec: expr=[sum@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a@0 + b@1 as sum] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[sum@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec ProjectionExec: expr=[a@0 + b@1 as sum] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2391,13 +2391,13 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortRequiredExec: [c@2 ASC] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2428,13 +2428,13 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2454,28 +2454,28 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2504,8 +2504,8 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] @@ -2513,13 +2513,13 @@ SortPreservingMergeExec: [a@0 ASC] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] @@ -2527,7 +2527,7 @@ SortExec: expr=[a@0 ASC], preserve_partitioning=[false] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -2546,28 +2546,28 @@ fn parallelization_single_partition() -> Result<()> { let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2598,13 +2598,13 @@ fn parallelization_multiple_files() -> Result<()> { let plan_3_distrib = test_config_concurrency_3.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_3_distrib, - @r" + plan_3_distrib, + @r" SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); let plan_3_sort = test_config_concurrency_3.run(plan.clone(), &SORT_DISTRIB_DISTRIB); assert_plan!(plan_3_distrib, plan_3_sort); @@ -2612,13 +2612,13 @@ SortRequiredExec: [a@0 ASC] let plan_8_distrib = test_config_concurrency_8.run(plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_8_distrib, - @r" + plan_8_distrib, + @r" SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); let plan_8_sort = test_config_concurrency_8.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_8_distrib, plan_8_sort); @@ -2638,54 +2638,54 @@ fn parallelization_compressed_csv() -> Result<()> { ]; insta::allow_duplicates! { - for compression_type in compression_types { - let plan = aggregate_exec_with_alias( - DataSourceExec::from_data_source( - FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_file_compression_type(compression_type) - .build(), - ), - vec![("a".to_string(), "a".to_string())], - ); - let test_config = TestConfig::default() - .with_query_execution_partitions(2) - .with_prefer_repartition_file_scans(10); - - let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - if compression_type.is_compressed() { - // Compressed files cannot be partitioned - assert_plan!( - plan_distrib, - @r" + for compression_type in compression_types { + let plan = aggregate_exec_with_alias( + DataSourceExec::from_data_source( + FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_file_compression_type(compression_type) + .build(), + ), + vec![("a".to_string(), "a".to_string())], + ); + let test_config = TestConfig::default() + .with_query_execution_partitions(2) + .with_prefer_repartition_file_scans(10); + + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); + if compression_type.is_compressed() { + // Compressed files cannot be partitioned + assert_plan!( + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); - } else { - // Uncompressed files can be partitioned - assert_plan!( - plan_distrib, - @r" + ); + } else { + // Uncompressed files can be partitioned + assert_plan!( + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); - } + ); + } - let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); - assert_plan!(plan_distrib, plan_sort); + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); + } } - } Ok(()) } @@ -2703,28 +2703,28 @@ fn parallelization_two_partitions() -> Result<()> { let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2746,14 +2746,14 @@ fn parallelization_two_partitions_into_four() -> Result<()> { test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2761,14 +2761,14 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2794,14 +2794,14 @@ fn parallelization_sorted_limit() -> Result<()> { // data is sorted so can't repartition here // Doesn't parallelize for SortExec without preserve_partitioning assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2810,14 +2810,14 @@ GlobalLimitExec: skip=0, fetch=100 // data is sorted so can't repartition here // Doesn't parallelize for SortExec without preserve_partitioning assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2845,8 +2845,8 @@ fn parallelization_limit_with_filter() -> Result<()> { // ordering is not used in subsequent stages anyway. // SortExec doesn't benefit from input partitioning assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec LocalLimitExec: fetch=100 @@ -2855,7 +2855,7 @@ GlobalLimitExec: skip=0, fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2865,8 +2865,8 @@ GlobalLimitExec: skip=0, fetch=100 // ordering is not used in subsequent stages anyway. // SortExec doesn't benefit from input partitioning assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec LocalLimitExec: fetch=100 @@ -2875,7 +2875,7 @@ GlobalLimitExec: skip=0, fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2954,8 +2954,8 @@ fn parallelization_union_inputs() -> Result<()> { test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Union doesn't benefit from input partitioning - no parallelism assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet @@ -2963,7 +2963,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2971,8 +2971,8 @@ UnionExec let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Union doesn't benefit from input partitioning - no parallelism assert_plan!( - plan_csv_distrib, - @r" + plan_csv_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false @@ -2980,7 +2980,7 @@ UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false " - ); + ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -3196,12 +3196,12 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> // Expected Outcome: // data should not be repartitioned / resorted assert_plan!( - plan_parquet_distrib, - @r" + plan_parquet_distrib, + @r" ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -3233,23 +3233,23 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { .into(); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); assert_plan!( - plan_csv, - @r" + plan_csv, + @r" SortPreservingMergeExec: [c2@1 ASC] ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false " - ); + ); let plan_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3262,26 +3262,26 @@ fn remove_redundant_roundrobins() -> Result<()> { let repartition = repartition_exec(repartition_exec(input)); let physical_plan = repartition_exec(filter_exec(repartition)); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3305,14 +3305,14 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3336,14 +3336,14 @@ fn preserve_ordering_through_repartition() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [d@3 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3365,28 +3365,28 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -3405,14 +3405,14 @@ fn no_need_for_sort_after_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3440,21 +3440,21 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!( - plan_sort, - @r" + plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -3462,7 +3462,7 @@ SortExec: expr=[a@0 ASC], preserve_partitioning=[false] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -3481,13 +3481,13 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3511,7 +3511,7 @@ SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; @@ -3526,7 +3526,7 @@ SortRequiredExec: [a@0 ASC] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -3549,7 +3549,7 @@ SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; @@ -3563,7 +3563,7 @@ SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet " - ); + ); Ok(()) } @@ -3586,13 +3586,13 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3618,8 +3618,8 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] @@ -3628,7 +3628,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3639,22 +3639,22 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] fn optimize_away_unnecessary_repartition() -> Result<()> { let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" CoalescePartitionsExec RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3667,8 +3667,8 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { filter_exec(repartition_exec(parquet_exec())), ))); assert_plan!( - physical_plan, - @r" + physical_plan, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 CoalescePartitionsExec @@ -3676,19 +3676,19 @@ FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); assert_plan!( - plan_distrib, - @r" + plan_distrib, + @r" FilterExec: c@2 = 0 FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3711,34 +3711,34 @@ async fn test_distribute_sort_parquet() -> Result<()> { // prior to optimization, this is the starting plan assert_plan!( - physical_plan, - @r" + physical_plan, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // what the enforce distribution run does. let plan_distribution = test_config.run(physical_plan.clone(), &[Run::Distribution]); assert_plan!( - plan_distribution, - @r" + plan_distribution, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); // what the sort parallelization (in enforce sorting), does after the enforce distribution changes let plan_both = test_config.run(physical_plan, &[Run::Distribution, Run::Sorting]); assert_plan!( - plan_both, - @r" + plan_both, + @r" SortPreservingMergeExec: [c@2 ASC] SortExec: expr=[c@2 ASC], preserve_partitioning=[true] DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet " - ); + ); Ok(()) } @@ -3764,13 +3764,13 @@ async fn test_distribute_sort_memtable() -> Result<()> { // this is the final, optimized plan assert_plan!( - physical_plan, - @r" + physical_plan, + @r" SortPreservingMergeExec: [id@0 ASC NULLS LAST] SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] DataSourceExec: partitions=3, partition_sizes=[34, 33, 33] " - ); + ); Ok(()) } From 2d67ef818c56d9808f4162ee087ff90d9265f056 Mon Sep 17 00:00:00 2001 From: blaginin Date: Tue, 21 Oct 2025 11:28:49 +0100 Subject: [PATCH 40/41] Fmt --- .../enforce_distribution.rs | 945 +++++++----------- 1 file changed, 385 insertions(+), 560 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index ec3a2f3768815..ed4a691391eba 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1219,22 +1219,22 @@ fn reorder_join_keys_to_left_input() -> Result<()> { JoinType::RightAnti, ]; - insta::allow_duplicates! { - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); - let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); + let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); - // The top joins' join key ordering is adjusted based on the children inputs. - let (captured_join_type, modified_plan) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); - assert_eq!(captured_join_type, join_type.to_string()); + // The top joins' join key ordering is adjusted based on the children inputs. + let (captured_join_type, modified_plan) = + hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); + assert_eq!(captured_join_type, join_type.to_string()); - insta::assert_snapshot!(modified_plan, @r" + insta::allow_duplicates! {insta::assert_snapshot!(modified_plan, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)] ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)] @@ -1253,10 +1253,8 @@ HashJoinExec: mode=Partitioned, join_type=..., on=[(AA@1, a1@5), (B@2, b1@6), (C RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - } +");} + } Ok(()) } @@ -1355,20 +1353,20 @@ fn reorder_join_keys_to_right_input() -> Result<()> { JoinType::RightAnti, ]; - insta::allow_duplicates! { - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); - - let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); - - // The top joins' join key ordering is adjusted based on the children inputs. - let (_, plan_str) = hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); - insta::assert_snapshot!(plan_str, @r" + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); + + let reordered = reorder_join_keys_to_inputs(top_join).unwrap(); + + // The top joins' join key ordering is adjusted based on the children inputs. + let (_, plan_str) = + hide_first(reordered.as_ref(), r"join_type=(\w+)", "join_type=..."); + insta::allow_duplicates! {insta::assert_snapshot!(plan_str, @r" HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)] ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C] HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)] @@ -1387,10 +1385,8 @@ HashJoinExec: mode=Partitioned, join_type=..., on=[(C@3, c@2), (B@2, b1@6), (AA@ RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - } +");} + } Ok(()) } @@ -1426,28 +1422,30 @@ fn multi_smj_joins() -> Result<()> { Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, )]; - insta::allow_duplicates! { - for join_type in join_types { - let join = - sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - - // Top join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); - - let mut settings = Settings::clone_current(); - settings.add_filter(&format!("join_type={join_type}"), "join_type=..."); - settings.bind(|| { - let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + for join_type in join_types { + let join = + sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - assert_plan!(plan_distrib, @r" + // Top join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); + + let mut settings = Settings::clone_current(); + settings.add_filter(&format!("join_type={join_type}"), "join_type=..."); + + #[rustfmt::skip] + insta::allow_duplicates! { + settings.bind(|| { + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -1463,20 +1461,19 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test - // cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => { - assert_plan!(plan_distrib, @r" +"); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test + // cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 @@ -1494,18 +1491,17 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - } +"); + } + } - let plan_sort = test_config.run(top_join.clone(), &SORT_DISTRIB_DISTRIB); + let plan_sort = test_config.run(top_join.clone(), &SORT_DISTRIB_DISTRIB); - match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC @@ -1521,21 +1517,20 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition and Roundrobin repartition after - // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" +"); + } + // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition and Roundrobin repartition after + // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1555,29 +1550,27 @@ SortMergeJoin: join_type=..., on=[(a@0, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - } +"); + } + } + + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); + + let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); match join_type { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); - - let plan_distrib = test_config.run(top_join.clone(), &DISTRIB_DISTRIB_SORT); - - match join_type { - // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs - JoinType::Inner | JoinType::Right => { - // TODO(wiedld): show different test result if enforce sorting first. - assert_plan!(plan_distrib, @r" + // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] @@ -1593,13 +1586,12 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs - JoinType::Left | JoinType::Full => { - // TODO(wiedld): show different test result if enforce sorting first. - assert_plan!(plan_distrib, @r" +"); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortExec: expr=[b1@6 ASC], preserve_partitioning=[true] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 @@ -1617,20 +1609,19 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - // this match arm cannot be reached - _ => unreachable!() - } - - let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); - - match join_type { - // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs - JoinType::Inner | JoinType::Right => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" +"); + } + // this match arm cannot be reached + _ => unreachable!() + } + + let plan_sort = test_config.run(top_join, &SORT_DISTRIB_DISTRIB); + + match join_type { + // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] SortMergeJoin: join_type=..., on=[(a@0, b1@1)] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC @@ -1646,13 +1637,12 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs - JoinType::Left | JoinType::Full => { - // TODO(wiedld): show different test result if enforce distribution first. - assert_plan!(plan_sort, @r" +"); + } + // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1672,19 +1662,17 @@ SortMergeJoin: join_type=..., on=[(b1@6, c@2)] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); - } - // this match arm cannot be reached - _ => unreachable!() - } +"); } - _ => {} + // this match arm cannot be reached + _ => unreachable!() } - }); + } + _ => {} } - } - + }); + } + } Ok(()) } @@ -1741,9 +1729,7 @@ fn smj_join_key_ordering() -> Result<()> { // Test: run EnforceDistribution, then EnforceSort. // Only two RepartitionExecs added let plan_distrib = test_config.run(join.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, @r" SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a1@0 as a3, b1@1 as b3] @@ -1760,14 +1746,11 @@ SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(join, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, @r" SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 @@ -1790,8 +1773,7 @@ SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)] AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); Ok(()) } @@ -1822,14 +1804,12 @@ fn merge_does_not_need_sort() -> Result<()> { // data is already sorted let test_config = TestConfig::default(); let plan_distrib = test_config.run(exec.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] CoalesceBatchesExec: target_batch_size=4096 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet -" - ); +"); // Test: result IS DIFFERENT, if EnforceSorting is run first: // @@ -1838,15 +1818,13 @@ SortPreservingMergeExec: [a@0 ASC] // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with // SortExec at the top. let plan_sort = test_config.run(exec, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec CoalesceBatchesExec: target_batch_size=4096 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet -" - ); +"); Ok(()) } @@ -1874,8 +1852,7 @@ fn union_to_interleave() -> Result<()> { // Only two RepartitionExecs added, no final RepartitionExec required let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, + assert_plan!(plan_distrib, @r" AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[] @@ -1890,8 +1867,7 @@ fn union_to_interleave() -> Result<()> { AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - " - ); + "); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -1923,8 +1899,7 @@ fn union_not_to_interleave() -> Result<()> { let test_config = TestConfig::default().with_prefer_existing_union(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, + assert_plan!(plan_distrib, @r" AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[] RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20 @@ -1940,8 +1915,7 @@ fn union_not_to_interleave() -> Result<()> { AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - " - ); + "); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -1955,16 +1929,14 @@ fn added_repartition_to_single_partition() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, + assert_plan!(plan_distrib, @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - " - ); + "); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -1978,8 +1950,7 @@ fn repartition_deepest_node() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, + assert_plan!(plan_distrib, @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 @@ -1987,8 +1958,7 @@ fn repartition_deepest_node() -> Result<()> { FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - " - ); + "); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2001,8 +1971,7 @@ fn repartition_unsorted_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, + assert_plan!(plan_distrib, @r" GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec @@ -2010,8 +1979,7 @@ fn repartition_unsorted_limit() -> Result<()> { FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - " - ); + "); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2030,15 +1998,13 @@ fn repartition_sorted_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2060,16 +2026,14 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortRequiredExec: [c@2 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2086,9 +2050,8 @@ fn repartition_ignores_limit() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] @@ -2101,8 +2064,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2115,17 +2077,15 @@ fn repartition_ignores_union() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2145,13 +2105,11 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2174,24 +2132,20 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [c@2 ASC] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); Ok(()) } @@ -2213,28 +2167,24 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [c@2 ASC] UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); // test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); Ok(()) } @@ -2259,15 +2209,13 @@ fn repartition_does_not_destroy_sort() -> Result<()> { let test_config = TestConfig::default().with_prefer_existing_sort(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortRequiredExec: [d@3 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2300,17 +2248,15 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" UnionExec SortRequiredExec: [c@2 ASC] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2339,29 +2285,25 @@ fn repartition_transitively_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [sum@0 ASC] SortExec: expr=[sum@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a@0 + b@1 as sum] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, + @r" SortExec: expr=[sum@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec ProjectionExec: expr=[a@0 + b@1 as sum] RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); Ok(()) } @@ -2390,14 +2332,12 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortRequiredExec: [c@2 ASC] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2427,14 +2367,12 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -2453,29 +2391,25 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); Ok(()) } @@ -2503,31 +2437,27 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); Ok(()) } @@ -2545,29 +2475,25 @@ fn parallelization_single_partition() -> Result<()> { // Test: with parquet let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_parquet_distrib, - @r" + assert_plan!(plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_csv_distrib, - @r" + assert_plan!(plan_csv_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false -" - ); +"); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2597,28 +2523,24 @@ fn parallelization_multiple_files() -> Result<()> { test_config.clone().with_query_execution_partitions(3); let plan_3_distrib = test_config_concurrency_3.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_3_distrib, - @r" + assert_plan!(plan_3_distrib, + @r" SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet -" - ); +"); let plan_3_sort = test_config_concurrency_3.run(plan.clone(), &SORT_DISTRIB_DISTRIB); assert_plan!(plan_3_distrib, plan_3_sort); let test_config_concurrency_8 = test_config.with_query_execution_partitions(8); let plan_8_distrib = test_config_concurrency_8.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_8_distrib, - @r" + assert_plan!(plan_8_distrib, + @r" SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet -" - ); +"); let plan_8_sort = test_config_concurrency_8.run(plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_8_distrib, plan_8_sort); @@ -2637,55 +2559,52 @@ fn parallelization_compressed_csv() -> Result<()> { FileCompressionType::UNCOMPRESSED, ]; + #[rustfmt::skip] insta::allow_duplicates! { - for compression_type in compression_types { - let plan = aggregate_exec_with_alias( - DataSourceExec::from_data_source( - FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_file_compression_type(compression_type) - .build(), - ), - vec![("a".to_string(), "a".to_string())], - ); - let test_config = TestConfig::default() - .with_query_execution_partitions(2) - .with_prefer_repartition_file_scans(10); - - let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); - if compression_type.is_compressed() { - // Compressed files cannot be partitioned - assert_plan!( - plan_distrib, - @r" + for compression_type in compression_types { + let plan = aggregate_exec_with_alias( + DataSourceExec::from_data_source( + FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_file_compression_type(compression_type) + .build(), + ), + vec![("a".to_string(), "a".to_string())], + ); + let test_config = TestConfig::default() + .with_query_execution_partitions(2) + .with_prefer_repartition_file_scans(10); + + let plan_distrib = test_config.run(plan.clone(), &DISTRIB_DISTRIB_SORT); + if compression_type.is_compressed() { + // Compressed files cannot be partitioned + assert_plan!(plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false -" - ); - } else { - // Uncompressed files can be partitioned - assert_plan!( - plan_distrib, - @r" +"); + } else { + // Uncompressed files can be partitioned + assert_plan!(plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false -" - ); - } - - let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); - assert_plan!(plan_distrib, plan_sort); - } +"); } + + let plan_sort = test_config.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); + } + } Ok(()) } @@ -2702,29 +2621,24 @@ fn parallelization_two_partitions() -> Result<()> { // Test: with parquet let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_parquet_distrib, - @r" + assert_plan!(plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_csv_distrib, - @r" + assert_plan!(plan_csv_distrib, @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false -" - ); +"); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2745,30 +2659,25 @@ fn parallelization_two_partitions_into_four() -> Result<()> { let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions - assert_plan!( - plan_parquet_distrib, - @r" + assert_plan!(plan_parquet_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Multiple source files split across partitions - assert_plan!( - plan_csv_distrib, - @r" + assert_plan!(plan_csv_distrib, @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false -" - ); +"); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2793,15 +2702,12 @@ fn parallelization_sorted_limit() -> Result<()> { test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // data is sorted so can't repartition here // Doesn't parallelize for SortExec without preserve_partitioning - assert_plan!( - plan_parquet_distrib, - @r" + assert_plan!(plan_parquet_distrib, @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2809,15 +2715,13 @@ GlobalLimitExec: skip=0, fetch=100 let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // data is sorted so can't repartition here // Doesn't parallelize for SortExec without preserve_partitioning - assert_plan!( - plan_csv_distrib, - @r" + assert_plan!(plan_csv_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false -" - ); +"); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2844,9 +2748,8 @@ fn parallelization_limit_with_filter() -> Result<()> { // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. // SortExec doesn't benefit from input partitioning - assert_plan!( - plan_parquet_distrib, - @r" + assert_plan!(plan_parquet_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec LocalLimitExec: fetch=100 @@ -2854,8 +2757,7 @@ GlobalLimitExec: skip=0, fetch=100 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -2864,9 +2766,8 @@ GlobalLimitExec: skip=0, fetch=100 // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. // SortExec doesn't benefit from input partitioning - assert_plan!( - plan_csv_distrib, - @r" + assert_plan!(plan_csv_distrib, + @r" GlobalLimitExec: skip=0, fetch=100 CoalescePartitionsExec LocalLimitExec: fetch=100 @@ -2874,8 +2775,7 @@ GlobalLimitExec: skip=0, fetch=100 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false -" - ); +"); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2897,8 +2797,7 @@ fn parallelization_ignores_limit() -> Result<()> { // Test: with parquet let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_parquet_distrib, + assert_plan!(plan_parquet_distrib, @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 @@ -2912,15 +2811,13 @@ fn parallelization_ignores_limit() -> Result<()> { GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet - " - ); + "); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_csv_distrib, + assert_plan!(plan_csv_distrib, @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 @@ -2934,8 +2831,7 @@ fn parallelization_ignores_limit() -> Result<()> { GlobalLimitExec: skip=0, fetch=100 LocalLimitExec: fetch=100 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false - " - ); + "); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -2953,34 +2849,30 @@ fn parallelization_union_inputs() -> Result<()> { let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Union doesn't benefit from input partitioning - no parallelism - assert_plan!( - plan_parquet_distrib, - @r" + assert_plan!(plan_parquet_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); // Union doesn't benefit from input partitioning - no parallelism - assert_plan!( - plan_csv_distrib, - @r" + assert_plan!(plan_csv_distrib, + @r" UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false -" - ); +"); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -3011,8 +2903,7 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // Test: with parquet let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_parquet_distrib, + assert_plan!(plan_parquet_distrib, @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet" ); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); @@ -3020,8 +2911,7 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_csv_distrib, + assert_plan!(plan_csv_distrib, @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false" ); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); @@ -3057,49 +2947,41 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { // Test: with parquet let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_parquet_distrib, + assert_plan!(plan_parquet_distrib, @r" SortPreservingMergeExec: [c@2 ASC] UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet - " - ); + "); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_parquet_sort, + assert_plan!(plan_parquet_sort, @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet - " - ); + "); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_csv_distrib, + assert_plan!(plan_csv_distrib, @r" SortPreservingMergeExec: [c@2 ASC] UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false - " - ); + "); let plan_csv_sort = test_config.run(plan_csv.clone(), &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_csv_sort, + assert_plan!(plan_csv_sort, @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec UnionExec DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false - " - ); + "); Ok(()) } @@ -3129,25 +3011,21 @@ fn parallelization_does_not_benefit() -> Result<()> { // Test: with parquet let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_parquet_distrib, + assert_plan!(plan_parquet_distrib, @r" SortRequiredExec: [c@2 ASC] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet - " - ); + "); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); // Test: with csv let plan_csv_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_csv_distrib, + assert_plan!(plan_csv_distrib, @r" SortRequiredExec: [c@2 ASC] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false - " - ); + "); let plan_csv_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_csv_distrib, plan_csv_sort); @@ -3181,27 +3059,23 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> let plan_parquet = sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); - assert_plan!( - plan_parquet, + assert_plan!(plan_parquet, @r" SortPreservingMergeExec: [c2@1 ASC] ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet - " - ); + "); let test_config = TestConfig::default(); let plan_parquet_distrib = test_config.run(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT); // Expected Outcome: // data should not be repartitioned / resorted - assert_plan!( - plan_parquet_distrib, - @r" + assert_plan!(plan_parquet_distrib, + @r" ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); let plan_parquet_sort = test_config.run(plan_parquet, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_parquet_distrib, plan_parquet_sort); @@ -3232,24 +3106,20 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { }] .into(); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); - assert_plan!( - plan_csv, - @r" + assert_plan!(plan_csv, + @r" SortPreservingMergeExec: [c2@1 ASC] ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false -" - ); +"); let test_config = TestConfig::default(); let plan_distrib = test_config.run(plan_csv.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" ProjectionExec: expr=[a@0 as a2, c@2 as c2] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false -" - ); +"); let plan_sort = test_config.run(plan_csv, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3261,27 +3131,23 @@ fn remove_redundant_roundrobins() -> Result<()> { let input = parquet_exec(); let repartition = repartition_exec(repartition_exec(input)); let physical_plan = repartition_exec(filter_exec(repartition)); - assert_plan!( - physical_plan, - @r" + assert_plan!(physical_plan, + @r" RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3304,15 +3170,13 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { let test_config = TestConfig::default().with_prefer_existing_sort(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3335,15 +3199,13 @@ fn preserve_ordering_through_repartition() -> Result<()> { let test_config = TestConfig::default().with_prefer_existing_sort(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [d@3 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3364,29 +3226,25 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet -" - ); +"); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet -" - ); +"); Ok(()) } @@ -3404,15 +3262,12 @@ fn no_need_for_sort_after_filter() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, @r" CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3439,30 +3294,26 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" SortPreservingMergeExec: [a@0 ASC] SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); // Test: result IS DIFFERENT, if EnforceSorting is run first: let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); - assert_plan!( - plan_sort, - @r" + assert_plan!(plan_sort, + @r" SortExec: expr=[a@0 ASC], preserve_partitioning=[false] CoalescePartitionsExec SortExec: expr=[a@0 ASC], preserve_partitioning=[true] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); Ok(()) } @@ -3480,14 +3331,12 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3510,8 +3359,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; @@ -3525,8 +3373,7 @@ SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); Ok(()) } @@ -3548,8 +3395,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet -" - ); +"); let mut config = ConfigOptions::new(); config.execution.target_partitions = 10; @@ -3562,8 +3408,7 @@ SortRequiredExec: [a@0 ASC] SortRequiredExec: [a@0 ASC] FilterExec: c@2 = 0 DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet -" - ); +"); Ok(()) } @@ -3585,14 +3430,12 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let test_config = TestConfig::default().with_query_execution_partitions(1); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3617,9 +3460,8 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { let test_config = TestConfig::default().with_query_execution_partitions(4); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] @@ -3627,8 +3469,7 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3638,23 +3479,19 @@ AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] #[test] fn optimize_away_unnecessary_repartition() -> Result<()> { let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); - assert_plan!( - physical_plan, - @r" + assert_plan!(physical_plan, + @r" CoalescePartitionsExec RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3666,29 +3503,25 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( filter_exec(repartition_exec(parquet_exec())), ))); - assert_plan!( - physical_plan, - @r" + assert_plan!(physical_plan, + @r" FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 CoalescePartitionsExec FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let test_config = TestConfig::default(); let plan_distrib = test_config.run(physical_plan.clone(), &DISTRIB_DISTRIB_SORT); - assert_plan!( - plan_distrib, - @r" + assert_plan!(plan_distrib, + @r" FilterExec: c@2 = 0 FilterExec: c@2 = 0 RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); let plan_sort = test_config.run(physical_plan, &SORT_DISTRIB_DISTRIB); assert_plan!(plan_distrib, plan_sort); @@ -3710,35 +3543,29 @@ async fn test_distribute_sort_parquet() -> Result<()> { let physical_plan = sort_exec(sort_key, parquet_exec_with_stats(10000 * 8192)); // prior to optimization, this is the starting plan - assert_plan!( - physical_plan, - @r" + assert_plan!(physical_plan, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); // what the enforce distribution run does. let plan_distribution = test_config.run(physical_plan.clone(), &[Run::Distribution]); - assert_plan!( - plan_distribution, - @r" + assert_plan!(plan_distribution, + @r" SortExec: expr=[c@2 ASC], preserve_partitioning=[false] CoalescePartitionsExec DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); // what the sort parallelization (in enforce sorting), does after the enforce distribution changes let plan_both = test_config.run(physical_plan, &[Run::Distribution, Run::Sorting]); - assert_plan!( - plan_both, - @r" + assert_plan!(plan_both, + @r" SortPreservingMergeExec: [c@2 ASC] SortExec: expr=[c@2 ASC], preserve_partitioning=[true] DataSourceExec: file_groups={10 groups: [[x:0..8192000], [x:8192000..16384000], [x:16384000..24576000], [x:24576000..32768000], [x:32768000..40960000], [x:40960000..49152000], [x:49152000..57344000], [x:57344000..65536000], [x:65536000..73728000], [x:73728000..81920000]]}, projection=[a, b, c, d, e], file_type=parquet -" - ); +"); Ok(()) } @@ -3763,14 +3590,12 @@ async fn test_distribute_sort_memtable() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; // this is the final, optimized plan - assert_plan!( - physical_plan, - @r" + assert_plan!(physical_plan, + @r" SortPreservingMergeExec: [id@0 ASC NULLS LAST] SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] DataSourceExec: partitions=3, partition_sizes=[34, 33, 33] -" - ); +"); Ok(()) } From 0b99eaf05eff60a860aca91bee4028ae75c1db8f Mon Sep 17 00:00:00 2001 From: blaginin Date: Mon, 3 Nov 2025 20:23:04 +0000 Subject: [PATCH 41/41] Update old methods --- .../enforce_distribution.rs | 634 ++++++++---------- 1 file changed, 282 insertions(+), 352 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index db011c4be43ab..5b7d9ac8fbe99 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -66,8 +66,8 @@ use datafusion_physical_plan::projection::{ProjectionExec, ProjectionExpr}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{ - displayable, get_plan_string, DisplayAs, DisplayFormatType, ExecutionPlanProperties, - PlanProperties, Statistics, + displayable, DisplayAs, DisplayFormatType, ExecutionPlanProperties, PlanProperties, + Statistics, }; use insta::Settings; @@ -469,83 +469,6 @@ impl TestConfig { self } - // This be deleted in https://github.com/apache/datafusion/pull/18185 - /// Perform a series of runs using the current [`TestConfig`], - /// assert the expected plan result, - /// and return the result plan (for potential subsequent runs). - fn run( - &self, - expected_lines: &[&str], - plan: Arc, - optimizers_to_run: &[Run], - ) -> Result> { - let expected_lines: Vec<&str> = expected_lines.to_vec(); - - // Add the ancillary output requirements operator at the start: - let optimizer = OutputRequirements::new_add_mode(); - let mut optimized = optimizer.optimize(plan.clone(), &self.config)?; - - // This file has 2 rules that use tree node, apply these rules to original plan consecutively - // After these operations tree nodes should be in a consistent state. - // This code block makes sure that these rules doesn't violate tree node integrity. - { - let adjusted = if self.config.optimizer.top_down_join_key_reordering { - // Run adjust_input_keys_ordering rule - let plan_requirements = - PlanWithKeyRequirements::new_default(plan.clone()); - let adjusted = plan_requirements - .transform_down(adjust_input_keys_ordering) - .data() - .and_then(check_integrity)?; - // TODO: End state payloads will be checked here. - adjusted.plan - } else { - // Run reorder_join_keys_to_inputs rule - plan.clone() - .transform_up(|plan| { - Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) - }) - .data()? - }; - - // Then run ensure_distribution rule - DistributionContext::new_default(adjusted) - .transform_up(|distribution_context| { - ensure_distribution(distribution_context, &self.config) - }) - .data() - .and_then(check_integrity)?; - // TODO: End state payloads will be checked here. - } - - for run in optimizers_to_run { - optimized = match run { - Run::Distribution => { - let optimizer = EnforceDistribution::new(); - optimizer.optimize(optimized, &self.config)? - } - Run::Sorting => { - let optimizer = EnforceSorting::new(); - optimizer.optimize(optimized, &self.config)? - } - }; - } - - // Remove the ancillary output requirements operator when done: - let optimizer = OutputRequirements::new_remove_mode(); - let optimized = optimizer.optimize(optimized, &self.config)?; - - // Now format correctly - let actual_lines = get_plan_string(&optimized); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{expected_lines:#?}\nactual:\n\n{actual_lines:#?}\n\n" - ); - - Ok(optimized) - } - /// Perform a series of runs using the current [`TestConfig`], /// assert the expected plan result, /// and return the result plan (for potential subsequent runs). @@ -1503,15 +1426,6 @@ fn multi_smj_joins() -> Result<()> { for join_type in join_types { let join = sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = |shift| -> String { - format!( - "{}SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]", - " ".repeat(shift) - ) - }; - let join_plan_indent2 = join_plan(2); - let join_plan_indent6 = join_plan(6); - let join_plan_indent10 = join_plan(10); // Top join on (a == c) let top_join_on = vec![( @@ -1520,235 +1434,246 @@ fn multi_smj_joins() -> Result<()> { )]; let top_join = sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); - let top_join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]"); - - let expected = match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => - vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test - // cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => vec![ - top_join_plan.as_str(), - // Below 2 operators are differences introduced, when join mode is changed - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - &join_plan_indent6, - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - }; - // TODO(wiedld): show different test result if enforce sorting first. - test_config.run(&expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - - let expected_first_sort_enforcement = match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => - vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition and Roundrobin repartition after - // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => vec![ - top_join_plan.as_str(), - // Below 4 operators are differences introduced, when join mode is changed - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - &join_plan_indent10, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - }; - // TODO(wiedld): show different test result if enforce distribution first. - test_config.run( - &expected_first_sort_enforcement, - top_join, - &SORT_DISTRIB_DISTRIB, - )?; - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = - sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); - let top_join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]"); - - let expected = match join_type { - // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs - JoinType::Inner | JoinType::Right => vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs - JoinType::Left | JoinType::Full => vec![ - top_join_plan.as_str(), - " SortExec: expr=[b1@6 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", - &join_plan_indent6, - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // this match arm cannot be reached - _ => unreachable!() - }; - // TODO(wiedld): show different test result if enforce sorting first. - test_config.run(&expected, top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - - let expected_first_sort_enforcement = match join_type { - // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs - JoinType::Inner | JoinType::Right => vec![ - top_join_plan.as_str(), - &join_plan_indent2, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs - JoinType::Left | JoinType::Full => vec![ - top_join_plan.as_str(), - " RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@6 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - &join_plan_indent10, - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - ], - // this match arm cannot be reached - _ => unreachable!() - }; + let mut settings = Settings::clone_current(); + settings.add_filter(&format!("join_type={join_type}"), "join_type=..."); - // TODO(wiedld): show different test result if enforce distribution first. - test_config.run( - &expected_first_sort_enforcement, - top_join, - &SORT_DISTRIB_DISTRIB, - )?; - } - _ => {} + #[rustfmt::skip] + insta::allow_duplicates! { + settings.bind(|| { + let plan_distrib = test_config.to_plan(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + assert_plan!(plan_distrib, @r" +SortMergeJoin: join_type=..., on=[(a@0, c@2)] + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +"); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test + // cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + assert_plan!(plan_distrib, @r" +SortMergeJoin: join_type=..., on=[(a@0, c@2)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +"); + } + } + + let plan_sort = test_config.to_plan(top_join.clone(), &SORT_DISTRIB_DISTRIB); + + match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" +SortMergeJoin: join_type=..., on=[(a@0, c@2)] + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +"); + } + // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition and Roundrobin repartition after + // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" +SortMergeJoin: join_type=..., on=[(a@0, c@2)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +"); + } + } + + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); + + let plan_distrib = test_config.to_plan(top_join.clone(), &DISTRIB_DISTRIB_SORT); + + match join_type { + // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" +SortMergeJoin: join_type=..., on=[(b1@6, c@2)] + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +"); + } + // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce sorting first. + assert_plan!(plan_distrib, @r" +SortMergeJoin: join_type=..., on=[(b1@6, c@2)] + SortExec: expr=[b1@6 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + SortExec: expr=[a@0 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[b1@1 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + SortExec: expr=[c@2 ASC], preserve_partitioning=[true] + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10 + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +"); + } + // this match arm cannot be reached + _ => unreachable!() + } + + let plan_sort = test_config.to_plan(top_join, &SORT_DISTRIB_DISTRIB); + + match join_type { + // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs + JoinType::Inner | JoinType::Right => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" +SortMergeJoin: join_type=..., on=[(b1@6, c@2)] + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +"); + } + // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs + JoinType::Left | JoinType::Full => { + // TODO(wiedld): show different test result if enforce distribution first. + assert_plan!(plan_sort, @r" +SortMergeJoin: join_type=..., on=[(b1@6, c@2)] + RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@6 ASC], preserve_partitioning=[false] + CoalescePartitionsExec + SortMergeJoin: join_type=..., on=[(a@0, b1@1)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[a@0 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[b1@1 ASC], preserve_partitioning=[false] + ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC + RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1 + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet +"); + } + // this match arm cannot be reached + _ => unreachable!() + } + } + _ => {} + } + }); } } - Ok(()) } @@ -2667,46 +2592,51 @@ fn parallelization_compressed_csv() -> Result<()> { FileCompressionType::UNCOMPRESSED, ]; - let expected_not_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - - let expected_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; + #[rustfmt::skip] + insta::allow_duplicates! { + for compression_type in compression_types { + let plan = aggregate_exec_with_alias( + DataSourceExec::from_data_source( + FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_file_compression_type(compression_type) + .build(), + ), + vec![("a".to_string(), "a".to_string())], + ); + let test_config = TestConfig::default() + .with_query_execution_partitions(2) + .with_prefer_repartition_file_scans(10); + + let plan_distrib = test_config.to_plan(plan.clone(), &DISTRIB_DISTRIB_SORT); + if compression_type.is_compressed() { + // Compressed files cannot be partitioned + assert_plan!(plan_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +"); + } else { + // Uncompressed files can be partitioned + assert_plan!(plan_distrib, + @r" +AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[] + RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 + AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] + DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false +"); + } - for compression_type in compression_types { - let expected = if compression_type.is_compressed() { - &expected_not_partitioned[..] - } else { - &expected_partitioned[..] - }; - - let plan = aggregate_exec_with_alias( - DataSourceExec::from_data_source( - FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_file_compression_type(compression_type) - .build(), - ), - vec![("a".to_string(), "a".to_string())], - ); - let test_config = TestConfig::default() - .with_query_execution_partitions(2) - .with_prefer_repartition_file_scans(10); - test_config.run(expected, plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, plan, &SORT_DISTRIB_DISTRIB)?; + let plan_sort = test_config.to_plan(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); + } } Ok(()) }