diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 63111f43806b..ed4a691391eb 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -66,9 +66,52 @@ 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, - 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) { + 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!( + displayable($plan.as_ref()).indent(true).to_string(), + @ $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 /// ordering but is easy to construct @@ -352,22 +395,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(); @@ -445,14 +472,11 @@ impl TestConfig { /// 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( + fn try_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)?; @@ -507,30 +531,16 @@ impl TestConfig { 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) } -} - -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 - ); - }; + fn run( + &self, + plan: Arc, + optimizers_to_run: &[Run], + ) -> Arc { + self.try_run(plan, optimizers_to_run).unwrap() + } } #[test] @@ -556,6 +566,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 _, @@ -564,11 +576,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 @@ -589,50 +607,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_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 => 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_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 + 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_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)] + 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.run(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); } JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => {} } + + match join_type { JoinType::Inner | JoinType::Left @@ -650,55 +676,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_distrib = test_config.run(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_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 + 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_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)] + 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.run(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); } JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} } + + }); + } } Ok(()) @@ -737,23 +772,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.run(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.run(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); // Join on (a2 == c) let top_join_on = vec![( @@ -764,23 +803,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.run(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.run(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -816,26 +859,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.run(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.run(top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -861,22 +907,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.run(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.run(join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -914,23 +964,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.run(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.run(join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1034,30 +1088,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.run(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.run(filter_top_join, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1168,34 +1226,34 @@ fn reorder_join_keys_to_left_input() -> Result<()> { &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); - 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", - ]; - - assert_plan_txt!(expected, reordered); + 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::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)] + 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(()) @@ -1302,34 +1360,32 @@ fn reorder_join_keys_to_right_input() -> Result<()> { &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)?; + 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)]", - " 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); + 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)] + 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(()) @@ -1369,15 +1425,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![( @@ -1386,235 +1433,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=..."); + + #[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] + 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.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" +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.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] + 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.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 + 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(()) } @@ -1670,52 +1728,52 @@ 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.run(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.run(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(()) } @@ -1744,13 +1802,14 @@ 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.run(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: // @@ -1758,13 +1817,14 @@ 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.run(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(()) } @@ -1790,25 +1850,26 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1834,28 +1895,29 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1865,17 +1927,18 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1885,18 +1948,19 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1905,19 +1969,19 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1932,17 +1996,17 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1960,19 +2024,18 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -1985,26 +2048,25 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2013,19 +2075,19 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2041,15 +2103,15 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2068,24 +2130,22 @@ 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.run(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.run(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(()) } @@ -2105,27 +2165,26 @@ 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.run(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.run(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(()) } @@ -2149,16 +2208,16 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2187,22 +2246,19 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2227,28 +2283,27 @@ 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.run(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.run(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(()) } @@ -2275,16 +2330,16 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2310,16 +2365,16 @@ 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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -2334,28 +2389,27 @@ 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.run(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.run(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(()) } @@ -2381,30 +2435,29 @@ 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.run(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.run(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(()) } @@ -2420,28 +2473,29 @@ 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.run(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.run(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.run(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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2465,40 +2519,30 @@ 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.run(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.run(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.run(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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_8_distrib, plan_8_sort); Ok(()) } @@ -2515,46 +2559,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.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" +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.run(plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); + } } Ok(()) } @@ -2570,30 +2619,28 @@ 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.run(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.run(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.run(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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2609,30 +2656,30 @@ 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.run(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.run(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.run(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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2651,32 +2698,32 @@ 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.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" +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); // 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.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" +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); Ok(()) } @@ -2696,40 +2743,41 @@ 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.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" +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.run(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.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 + 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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2747,48 +2795,45 @@ 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.run(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.run(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.run(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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2801,34 +2846,35 @@ 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.run(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.run(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.run(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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2855,22 +2901,21 @@ 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.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.run(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.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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -2900,54 +2945,43 @@ 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.run(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.run(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.run(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.run(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(()) } @@ -2975,24 +3009,25 @@ 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.run(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.run(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.run(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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_csv_distrib, plan_csv_sort); Ok(()) } @@ -3023,26 +3058,26 @@ 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 + "); + + 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 - 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.run(plan_parquet, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_parquet_distrib, plan_parquet_sort); Ok(()) } @@ -3071,22 +3106,22 @@ 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 +"); - // 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.run(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.run(plan_csv, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3096,24 +3131,25 @@ 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); - - 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", - ]; + 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(); - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + let plan_distrib = test_config.run(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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3133,18 +3169,16 @@ 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.run(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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3164,14 +3198,16 @@ 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.run(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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3189,29 +3225,26 @@ 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.run(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.run(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(()) } @@ -3227,17 +3260,16 @@ 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.run(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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3261,30 +3293,27 @@ 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.run(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.run(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(()) } @@ -3300,14 +3329,16 @@ 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.run(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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3322,30 +3353,27 @@ 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(()) } @@ -3361,29 +3389,26 @@ 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(()) } @@ -3404,13 +3429,15 @@ 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.run(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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3432,19 +3459,19 @@ 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.run(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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3452,19 +3479,21 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { #[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()); - - let expected = - &["DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet"]; + 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(); - test_config.run(expected, physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; - test_config.run(expected, physical_plan, &SORT_DISTRIB_DISTRIB)?; + 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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3474,25 +3503,27 @@ 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 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.run(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.run(physical_plan, &SORT_DISTRIB_DISTRIB); + assert_plan!(plan_distrib, plan_sort); Ok(()) } @@ -3512,27 +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 - 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.run(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.run(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(()) } @@ -3557,12 +3590,12 @@ 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(()) }