diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 63111f43806b..6fcf858c2177 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use insta::{allow_duplicates, assert_snapshot, with_settings}; use std::fmt::Debug; use std::ops::Deref; use std::sync::Arc; @@ -66,8 +67,8 @@ use datafusion_physical_plan::projection::{ProjectionExec, ProjectionExpr}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{ - get_plan_string, DisplayAs, DisplayFormatType, ExecutionPlanProperties, - PlanProperties, Statistics, + displayable, DisplayAs, DisplayFormatType, ExecutionPlanProperties, PlanProperties, + Statistics, }; /// Models operators like BoundedWindowExec that require an input @@ -352,22 +353,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 +430,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 run_and_get_plan_string( &self, - expected_lines: &[&str], plan: Arc, optimizers_to_run: &[Run], - ) -> Result> { - let expected_lines: Vec<&str> = expected_lines.to_vec(); - + ) -> Result { // Add the ancillary output requirements operator at the start: let optimizer = OutputRequirements::new_add_mode(); let mut optimized = optimizer.optimize(plan.clone(), &self.config)?; @@ -508,31 +490,12 @@ impl TestConfig { 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" - ); + let actual_optimized = displayable(optimized.as_ref()).indent(true).to_string(); - Ok(optimized) + Ok(actual_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 - ); - }; -} - #[test] fn multi_hash_joins() -> Result<()> { let left = parquet_exec(); @@ -564,11 +527,6 @@ 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); match join_type { JoinType::Inner @@ -589,46 +547,117 @@ 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 test_config = TestConfig::default(); + let actual_1 = test_config + .run_and_get_plan_string(top_join.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config + .run_and_get_plan_string(top_join, &SORT_DISTRIB_DISTRIB)?; - let expected = match join_type { + 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 => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_1, + @r" + HashJoinExec: mode=Partitioned, join_type=[type], on=[(a@0, c@2)] + HashJoinExec: mode=Partitioned, join_type=[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 + " + ); + }); + + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_2, + @r" + HashJoinExec: mode=Partitioned, join_type=[type], on=[(a@0, c@2)] + HashJoinExec: mode=Partitioned, join_type=[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", - ], + _ => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_1, + @r" + HashJoinExec: mode=Partitioned, join_type=[type], on=[(a@0, c@2)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + HashJoinExec: mode=Partitioned, join_type=[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 + " + ); + }); + + + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_2, + @r" + HashJoinExec: mode=Partitioned, join_type=[type], on=[(a@0, c@2)] + RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10 + HashJoinExec: mode=Partitioned, join_type=[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)?; } JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => {} } @@ -650,52 +679,148 @@ 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 actual_1 = test_config + .run_and_get_plan_string(top_join.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config + .run_and_get_plan_string(top_join, &SORT_DISTRIB_DISTRIB)?; + + // Split the plan string into the first line and the rest + let (first_line_1, other_lines) = + actual_1.split_once('\n').unwrap_or((&actual_1, "")); + let (first_line_2, other_lines_2) = + actual_2.split_once('\n').unwrap_or((&actual_2, "")); + + match join_type { + JoinType::RightSemi | JoinType::RightAnti => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + first_line_1, + @"HashJoinExec: mode=Partitioned, join_type=[type], on=[(b1@1, c@2)]" + ); + }); + } + } + _ => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + first_line_2, + @"HashJoinExec: mode=Partitioned, join_type=[type], on=[(b1@6, c@2)]" + ); + }); + } + } + } + + 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 + | JoinType::RightSemi + | JoinType::RightAnti => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + other_lines, + @r" + HashJoinExec: mode=Partitioned, join_type=[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 + " + ); + }); + } + + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + other_lines_2, + @r" + HashJoinExec: mode=Partitioned, join_type=[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", - ], + _ => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + other_lines, + @r" + RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 + HashJoinExec: mode=Partitioned, join_type=[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 + " + ); + }); + } + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + other_lines_2, + @r" + RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10 + HashJoinExec: mode=Partitioned, join_type=[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)?; } JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} } @@ -737,23 +862,33 @@ 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 actual_1 = + test_config.run_and_get_plan_string(top_join.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(top_join, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // Join on (a2 == c) let top_join_on = vec![( @@ -764,23 +899,33 @@ 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 actual_1 = + test_config.run_and_get_plan_string(top_join.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(top_join, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -816,27 +961,59 @@ 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 actual_1 = + test_config.run_and_get_plan_string(top_join.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(top_join, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -861,23 +1038,31 @@ 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 actual_1 = + test_config.run_and_get_plan_string(join.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(join, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -914,24 +1099,32 @@ 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 actual_1 = + test_config.run_and_get_plan_string(join.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(join, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1034,30 +1227,40 @@ 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 actual_1 = test_config + .run_and_get_plan_string(filter_top_join.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(filter_top_join, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1168,34 +1371,40 @@ 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 actual = displayable(reordered.as_ref()).indent(true).to_string(); // 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); + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual, + @r" + HashJoinExec: mode=Partitioned, join_type=[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=[type], 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=[type], 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 +1511,39 @@ 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 actual = displayable(reordered.as_ref()).indent(true).to_string(); // 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); + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual, + @r" + HashJoinExec: mode=Partitioned, join_type=[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=[type], 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=[type], 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 +1583,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,120 +1591,164 @@ 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 actual_1 = test_config + .run_and_get_plan_string(top_join.clone(), &DISTRIB_DISTRIB_SORT)?; - let expected_first_sort_enforcement = match join_type { + 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", - ], + JoinType::Inner + | JoinType::Left + | JoinType::LeftSemi + | JoinType::LeftAnti => { + // 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 + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_1, + @r" + SortMergeJoin: join_type=[type], on=[(a@0, c@2)] + SortMergeJoin: join_type=[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 + " + ); + }); + } + } + _ => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_1, + @r" + SortMergeJoin: join_type=[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=[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 actual_2 = test_config + .run_and_get_plan_string(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 => { + // 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 + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_2, + @r" + SortMergeJoin: join_type=[type], on=[(a@0, c@2)] + SortMergeJoin: join_type=[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 + " + ); + }); + } + } + _ => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_2, + @r" + SortMergeJoin: join_type=[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=[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 => { @@ -1511,105 +1760,146 @@ fn multi_smj_joins() -> Result<()> { )]; 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 { + // TODO(wiedld): show different test result if enforce sorting first. + let actual_1 = test_config + .run_and_get_plan_string(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 => 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", - ], + JoinType::Inner | JoinType::Right => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_1, + @r" + SortMergeJoin: join_type=[type], on=[(b1@6, c@2)] + SortMergeJoin: join_type=[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 => 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", - ], + JoinType::Left | JoinType::Full => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_1, + @r" + SortMergeJoin: join_type=[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=[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!() + _ => 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 { + // TODO(wiedld): show different test result if enforce distribution first. + let actual_2 = test_config + .run_and_get_plan_string(top_join.clone(), &SORT_DISTRIB_DISTRIB)?; + + 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", - ], + JoinType::Inner | JoinType::Right => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_2, + @r" + SortMergeJoin: join_type=[type], on=[(b1@6, c@2)] + SortMergeJoin: join_type=[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 => 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", - ], + JoinType::Left | JoinType::Full => { + allow_duplicates! { + with_settings!({ + filters => vec![(r"join_type=\w+", "join_type=[type]")] + }, { + assert_snapshot!( + actual_2, + @r" + SortMergeJoin: join_type=[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=[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!() + _ => unreachable!(), }; - - // TODO(wiedld): show different test result if enforce distribution first. - test_config.run( - &expected_first_sort_enforcement, - top_join, - &SORT_DISTRIB_DISTRIB, - )?; } _ => {} } @@ -1670,53 +1960,60 @@ 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 actual_1 = + test_config.run_and_get_plan_string(join.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = test_config.run_and_get_plan_string(join, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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 +2041,18 @@ 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 actual_1 = + test_config.run_and_get_plan_string(exec.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 +2060,17 @@ 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 actual_2 = test_config.run_and_get_plan_string(exec, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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 +2096,33 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1833,29 +2147,37 @@ fn union_not_to_interleave() -> Result<()> { let plan = 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + // Only two RepartitionExecs added, no final RepartitionExec required + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1865,17 +2187,25 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1885,18 +2215,26 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1905,19 +2243,26 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1932,17 +2277,24 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1960,19 +2312,27 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + // We can use repartition here, ordering requirement by SortRequiredExec + // is still satisfied. + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -1984,27 +2344,32 @@ fn repartition_ignores_limit() -> Result<()> { limit_exec(filter_exec(limit_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=[]", - " 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2013,19 +2378,26 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2042,15 +2414,22 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @r" + SortExec: expr=[c@2 ASC], preserve_partitioning=[false] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet + " + ); + } + } Ok(()) } @@ -2072,21 +2451,29 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { // // 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 test_config = TestConfig::default(); + let actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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(()) } @@ -2108,25 +2495,32 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { // 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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(()) } @@ -2150,15 +2544,23 @@ fn repartition_does_not_destroy_sort() -> Result<()> { 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2189,20 +2591,26 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { // 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2228,28 +2636,33 @@ fn repartition_transitively_with_projection() -> Result<()> { 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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 +2688,24 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + // Since this projection is trivial, increasing parallelism is not beneficial + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2310,17 +2731,24 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + // Since this projection is trivial, increasing parallelism is not beneficial + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2335,28 +2763,34 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + // Expect repartition on the input to the sort (as it can benefit from additional parallelism) + assert_snapshot!( + actual_1, + @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 actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + // Expect repartition on the input of the filter (as it can benefit from additional parallelism) + assert_snapshot!( + actual_2, + @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(()) } @@ -2382,29 +2816,36 @@ 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + // Expect repartition on the input to the sort (as it can benefit from additional parallelism) + // repartition is lowest down + assert_snapshot!( + actual_1, + @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 actual_2 = test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + assert_snapshot!( + actual_2, + @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 +2861,44 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // 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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2465,40 +2922,44 @@ 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 actual_1 = test_config_concurrency_3 + .run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = test_config_concurrency_3 + .run_and_get_plan_string(plan.clone(), &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 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 actual_1 = test_config_concurrency_8 + .run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config_concurrency_8.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2515,28 +2976,7 @@ fn parallelization_compressed_csv() -> Result<()> { FileCompressionType::UNCOMPRESSED, ]; - let expected_not_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - - let expected_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - " RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - " DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - ]; - for compression_type in compression_types { - let expected = if compression_type.is_compressed() { - &expected_not_partitioned[..] - } else { - &expected_partitioned[..] - }; - let plan = aggregate_exec_with_alias( DataSourceExec::from_data_source( FileScanConfigBuilder::new( @@ -2553,8 +2993,44 @@ fn parallelization_compressed_csv() -> Result<()> { 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 actual_1 = + test_config.run_and_get_plan_string(plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan, &SORT_DISTRIB_DISTRIB)?; + + match compression_type { + FileCompressionType::UNCOMPRESSED => { + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } + } + } } Ok(()) } @@ -2570,31 +3046,44 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // 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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2609,30 +3098,45 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // 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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + // Multiple source files split across partitions + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2651,32 +3155,44 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // 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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2696,40 +3212,50 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // 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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2747,48 +3273,60 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // 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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2801,34 +3339,48 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // 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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -2855,23 +3407,34 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @"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 = &[ - "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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false" + ); + } + } Ok(()) } @@ -2900,55 +3463,60 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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 +3543,40 @@ 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 actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } // 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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3023,27 +3607,35 @@ 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); + let actual = displayable(plan_parquet.as_ref()).indent(true).to_string(); + assert_snapshot!( + actual, + @r" + SortPreservingMergeExec: [c2@1 ASC] + ProjectionExec: expr=[a@0 as a2, c@2 as c2] + DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet + " + ); // Expected Outcome: // data should not be repartitioned / resorted - 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)?; - + let actual_1 = test_config + .run_and_get_plan_string(plan_parquet.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_parquet, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3071,23 +3663,35 @@ 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); + let actual = displayable(plan_csv.as_ref()).indent(true).to_string(); + assert_snapshot!( + actual, + @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 actual_1 = + test_config.run_and_get_plan_string(plan_csv.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(plan_csv, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3096,24 +3700,36 @@ 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", - ]; + let actual = displayable(physical_plan.as_ref()).indent(true).to_string(); + assert_snapshot!( + actual, + @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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3136,15 +3752,24 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { // 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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3163,15 +3788,24 @@ 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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3190,28 +3824,34 @@ 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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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 +3867,27 @@ 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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. + // Since after this stage c is constant. c@2 ASC ordering is already satisfied. + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3262,29 +3912,35 @@ 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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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 +3956,30 @@ 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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + + assert_snapshot!( + actual_1, + @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 actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + assert_snapshot!( + actual_2, + @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 + " + ); Ok(()) } @@ -3322,30 +3994,35 @@ 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", - ]; + let actual = displayable(physical_plan.as_ref()).indent(true).to_string(); + // Ordering requirement of sort required exec is NOT satisfied + // by existing ordering at the source. + assert_snapshot!( + actual, + @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); + let actual = displayable(dist_plan.as_ref()).indent(true).to_string(); + // Since at the start of the rule ordering requirement is not satisfied + // EnforceDistribution rule doesn't satisfy this requirement either. + assert_snapshot!( + actual, + @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 +4038,34 @@ 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. + let actual = displayable(physical_plan.as_ref()).indent(true).to_string(); + assert_snapshot!( + actual, + @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); + let actual = displayable(dist_plan.as_ref()).indent(true).to_string(); + // Since at the start of the rule ordering requirement is satisfied + // EnforceDistribution rule satisfy this requirement also. + assert_snapshot!( + actual, + @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 +4086,23 @@ 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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3432,19 +4124,29 @@ 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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + // Since hash requirements of this operator is satisfied. There shouldn't be + // a hash repartition here + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3452,19 +4154,31 @@ 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"]; + let actual = displayable(physical_plan.as_ref()).indent(true).to_string(); + + assert_snapshot!( + actual, + @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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @"DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet" + ); + } + } Ok(()) } @@ -3474,25 +4188,38 @@ 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()); + let actual = displayable(physical_plan.as_ref()).indent(true).to_string(); + assert_snapshot!( + actual, + @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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &DISTRIB_DISTRIB_SORT)?; + let actual_2 = + test_config.run_and_get_plan_string(physical_plan, &SORT_DISTRIB_DISTRIB)?; + + for actual in [&actual_1, &actual_2] { + allow_duplicates! { + assert_snapshot!( + actual, + @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 + " + ); + } + } Ok(()) } @@ -3502,9 +4229,9 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { async fn test_distribute_sort_parquet() -> Result<()> { let test_config: TestConfig = TestConfig::default().with_prefer_repartition_file_scans(1000); - assert!( + assert_snapshot!( test_config.config.optimizer.repartition_file_scans, - "should enable scans to be repartitioned" + @"true" ); let schema = schema(); @@ -3512,27 +4239,41 @@ 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()); + let actual = displayable(physical_plan.as_ref()).indent(true).to_string(); + assert_snapshot!( + actual, + @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 actual_1 = test_config + .run_and_get_plan_string(physical_plan.clone(), &[Run::Distribution])?; + + assert_snapshot!( + actual_1, + @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 actual_2 = test_config + .run_and_get_plan_string(physical_plan, &[Run::Distribution, Run::Sorting])?; + + assert_snapshot!( + actual_2, + @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(()) } @@ -3541,9 +4282,9 @@ async fn test_distribute_sort_parquet() -> Result<()> { async fn test_distribute_sort_memtable() -> Result<()> { let test_config: TestConfig = TestConfig::default().with_prefer_repartition_file_scans(1000); - assert!( + assert_snapshot!( test_config.config.optimizer.repartition_file_scans, - "should enable scans to be repartitioned" + @"true" ); let mem_table = create_memtable()?; @@ -3557,12 +4298,16 @@ 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); + let actual = displayable(physical_plan.as_ref()).indent(true).to_string(); + + assert_snapshot!( + actual, + @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(()) } diff --git a/datafusion/core/tests/physical_optimizer/join_selection.rs b/datafusion/core/tests/physical_optimizer/join_selection.rs index 551fde5d7ff8..f9d3a045469e 100644 --- a/datafusion/core/tests/physical_optimizer/join_selection.rs +++ b/datafusion/core/tests/physical_optimizer/join_selection.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use insta::assert_snapshot; use std::sync::Arc; use std::{ any::Any, @@ -426,8 +427,7 @@ async fn test_join_with_swap_mark() { /// Compare the input plan with the plan after running the probe order optimizer. macro_rules! assert_optimized { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let expected_lines = $EXPECTED_LINES.iter().map(|s| *s).collect::>(); + ($PLAN: expr, @$EXPECTED_LINES: literal $(,)?) => { let plan = Arc::new($PLAN); let optimized = JoinSelection::new() @@ -435,12 +435,11 @@ macro_rules! assert_optimized { .unwrap(); let plan_string = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = plan_string.split("\n").collect::>(); + let actual = plan_string.trim(); - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines + assert_snapshot!( + actual, + @$EXPECTED_LINES ); }; } @@ -489,17 +488,18 @@ async fn test_nested_join_swap() { // The first hash join's left is 'small' table (with 1000 rows), and the second hash join's // left is the F(small IJ big) which has an estimated cardinality of 2000 rows (vs medium which // has an exact cardinality of 10_000 rows). - let expected = [ - "ProjectionExec: expr=[medium_col@2 as medium_col, big_col@0 as big_col, small_col@1 as small_col]", - " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]", - " ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", - " StatisticsExec: col_count=1, row_count=Inexact(1000)", - " StatisticsExec: col_count=1, row_count=Inexact(100000)", - " StatisticsExec: col_count=1, row_count=Inexact(10000)", - "", - ]; - assert_optimized!(expected, join); + assert_optimized!( + join, + @r" + ProjectionExec: expr=[medium_col@2 as medium_col, big_col@0 as big_col, small_col@1 as small_col] + HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)] + ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col] + HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)] + StatisticsExec: col_count=1, row_count=Inexact(1000) + StatisticsExec: col_count=1, row_count=Inexact(100000) + StatisticsExec: col_count=1, row_count=Inexact(10000) + " + ); } #[tokio::test] diff --git a/datafusion/core/tests/physical_optimizer/partition_statistics.rs b/datafusion/core/tests/physical_optimizer/partition_statistics.rs index 62ab5cbc422b..49dc5b845605 100644 --- a/datafusion/core/tests/physical_optimizer/partition_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/partition_statistics.rs @@ -17,6 +17,7 @@ #[cfg(test)] mod test { + use insta::assert_snapshot; use std::sync::Arc; use arrow::array::{Int32Array, RecordBatch}; @@ -606,21 +607,21 @@ mod test { .build() .map(Arc::new)?]; - let aggregate_exec_partial = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, - group_by.clone(), - aggr_expr.clone(), - vec![None], - Arc::clone(&scan), - scan_schema.clone(), - )?) as _; - - let mut plan_string = get_plan_string(&aggregate_exec_partial); - let _ = plan_string.swap_remove(1); - let expected_plan = vec![ - "AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]", - ]; - assert_eq!(plan_string, expected_plan); + let aggregate_exec_partial: Arc = + Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by.clone(), + aggr_expr.clone(), + vec![None], + Arc::clone(&scan), + scan_schema.clone(), + )?) as _; + + let plan_string = get_plan_string(&aggregate_exec_partial).swap_remove(0); + assert_snapshot!( + plan_string, + @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]" + ); let p0_statistics = aggregate_exec_partial.partition_statistics(Some(0))?; @@ -710,7 +711,10 @@ mod test { )?) as _; let agg_plan = get_plan_string(&agg_partial).remove(0); - assert_eq!("AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]",agg_plan); + assert_snapshot!( + agg_plan, + @"AggregateExec: mode=Partial, gby=[id@0 as id, 1 + id@0 as expr], aggr=[COUNT(c)]" + ); let empty_stat = Statistics { num_rows: Precision::Exact(0),