From 8ad047c5f7257445dab130a307c1eae49d89dd66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 3 Apr 2026 10:19:48 +0200 Subject: [PATCH 1/7] Preserve column order in projection embedding to eliminate redundant ProjectionExec nodes When `try_embed_projection` embeds a projection into a HashJoinExec (or other operators), `collect_column_indices` previously collected column indices into a HashSet and sorted them, losing the original column ordering from the projection. This meant that when a projection simply reordered columns, the embedded projection would output columns in sorted index order instead of the desired order, requiring a residual ProjectionExec just to reorder them. This change preserves the insertion order of column indices: - For simple Column expressions: the index is taken directly in projection order - For complex expressions referencing multiple columns: indices are sorted for determinism (since `collect_columns` returns a HashSet) Additionally, the final embedding decision now checks whether the embedding actually reduced the number of columns. If the residual ProjectionExec is not removable AND no columns were eliminated, the embedding is skipped (it would only add an unnecessary column reorder). Co-Authored-By: Claude Opus 4.6 (1M context) --- datafusion/physical-plan/src/projection.rs | 58 +++-- .../sqllogictest/test_files/clickbench.slt | 18 +- .../dynamic_filter_pushdown_config.slt | 56 ++--- .../sqllogictest/test_files/explain_tree.slt | 234 +++++++----------- .../sqllogictest/test_files/group_by.slt | 29 +-- .../sqllogictest/test_files/join.slt.part | 39 ++- .../test_files/join_is_not_distinct_from.slt | 12 +- .../test_files/join_limit_pushdown.slt | 18 +- datafusion/sqllogictest/test_files/joins.slt | 205 +++++++-------- .../sqllogictest/test_files/predicates.slt | 19 +- .../test_files/preserve_file_partitioning.slt | 44 ++-- .../test_files/projection_pushdown.slt | 12 +- .../repartition_subset_satisfaction.slt | 22 +- .../sqllogictest/test_files/subquery.slt | 33 ++- .../test_files/tpch/plans/q1.slt.part | 4 +- .../test_files/tpch/plans/q2.slt.part | 88 ++++--- .../test_files/tpch/plans/q7.slt.part | 4 +- .../test_files/tpch/plans/q8.slt.part | 4 +- .../test_files/tpch/plans/q9.slt.part | 4 +- testing | 2 +- 20 files changed, 418 insertions(+), 487 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index cd80277156fcb..75e9a568a79fe 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -607,13 +607,7 @@ pub fn try_embed_projection( return Ok(None); }; - // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. - // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of execution_plan schema fields. - if projection_index.len() == projection_index.last().unwrap() + 1 - && projection_index.len() == execution_plan.schema().fields().len() - { - return Ok(None); - } + let columns_reduced = projection_index.len() < execution_plan.schema().fields().len(); let new_execution_plan = Arc::new(execution_plan.with_projection(Some(projection_index.to_vec()))?); @@ -648,9 +642,16 @@ pub fn try_embed_projection( Arc::clone(&new_execution_plan) as _, )?); if is_projection_removable(&new_projection) { + // Residual is identity — embedding fully absorbed the projection. Ok(Some(new_execution_plan)) - } else { + } else if columns_reduced { + // Embedding reduced columns even though a residual is still needed + // for renames or expressions — worth keeping. Ok(Some(new_projection)) + } else { + // No columns eliminated and residual still needed — embedding just + // adds an unnecessary column reorder inside the operator. + Ok(None) } } @@ -1080,15 +1081,38 @@ fn try_unifying_projections( /// Collect all column indices from the given projection expressions. fn collect_column_indices(exprs: &[ProjectionExpr]) -> Vec { - // Collect indices and remove duplicates. - let mut indices = exprs - .iter() - .flat_map(|proj_expr| collect_columns(&proj_expr.expr)) - .map(|x| x.index()) - .collect::>() - .into_iter() - .collect::>(); - indices.sort(); + // Collect column indices in a deterministic order that preserves the + // projection's column ordering when possible. For simple Column + // expressions, we use the column index directly (preserving the + // projection's desired output order). For complex expressions with + // multiple column references, we sort indices for determinism since + // collect_columns returns a HashSet with non-deterministic iteration. + // This allows the embedded projection to match the desired output + // column order for simple column reorderings, avoiding a residual + // ProjectionExec. + let mut seen = std::collections::HashSet::new(); + let mut indices = Vec::new(); + for proj_expr in exprs { + if let Some(col) = proj_expr.expr.as_any().downcast_ref::() { + // Simple column reference: preserve projection order. + if seen.insert(col.index()) { + indices.push(col.index()); + } + } else { + // Complex expression: collect all referenced columns in sorted + // order for determinism. + let mut expr_indices: Vec = collect_columns(&proj_expr.expr) + .into_iter() + .map(|c| c.index()) + .collect(); + expr_indices.sort(); + for idx in expr_indices { + if seen.insert(idx) { + indices.push(idx); + } + } + } + } indices } diff --git a/datafusion/sqllogictest/test_files/clickbench.slt b/datafusion/sqllogictest/test_files/clickbench.slt index 4e9849e3650a8..314c3f9736e90 100644 --- a/datafusion/sqllogictest/test_files/clickbench.slt +++ b/datafusion/sqllogictest/test_files/clickbench.slt @@ -673,10 +673,9 @@ physical_plan 01)ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase] 02)--SortPreservingMergeExec: [EventTime@1 ASC NULLS LAST], fetch=10 03)----SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST], preserve_partitioning=[true] -04)------ProjectionExec: expr=[SearchPhrase@1 as SearchPhrase, EventTime@0 as EventTime] -05)--------FilterExec: SearchPhrase@1 != -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +04)------FilterExec: SearchPhrase@1 != , projection=[SearchPhrase@1, EventTime@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query T SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime" LIMIT 10; @@ -717,10 +716,9 @@ physical_plan 01)ProjectionExec: expr=[SearchPhrase@0 as SearchPhrase] 02)--SortPreservingMergeExec: [EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST], fetch=10 03)----SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------ProjectionExec: expr=[SearchPhrase@1 as SearchPhrase, EventTime@0 as EventTime] -05)--------FilterExec: SearchPhrase@1 != -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +04)------FilterExec: SearchPhrase@1 != , projection=[SearchPhrase@1, EventTime@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query T SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime", "SearchPhrase" LIMIT 10; @@ -1110,8 +1108,8 @@ physical_plan 05)--------AggregateExec: mode=FinalPartitioned, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] 06)----------RepartitionExec: partitioning=Hash([URLHash@0, EventDate@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[URLHash@0 as URLHash, EventDate@1 as EventDate], aggr=[count(Int64(1))] -08)--------------ProjectionExec: expr=[URLHash@1 as URLHash, CAST(CAST(EventDate@0 AS Int32) AS Date32) as EventDate] -09)----------------FilterExec: CounterID@1 = 62 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, projection=[EventDate@0, URLHash@5] +08)--------------ProjectionExec: expr=[URLHash@0 as URLHash, CAST(CAST(EventDate@1 AS Int32) AS Date32) as EventDate] +09)----------------FilterExec: CounterID@1 = 62 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@0 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@2 = 0 AND (TraficSourceID@3 = -1 OR TraficSourceID@3 = 6) AND RefererHash@4 = 3594120000172545465, projection=[URLHash@5, EventDate@0] 10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventDate, CounterID, IsRefresh, TraficSourceID, RefererHash, URLHash], file_type=parquet, predicate=CounterID@6 = 62 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) >= 2013-07-01 AND CAST(CAST(EventDate@5 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh@15 = 0 AND (TraficSourceID@37 = -1 OR TraficSourceID@37 = 6) AND RefererHash@102 = 3594120000172545465, pruning_predicate=CounterID_null_count@2 != row_count@3 AND CounterID_min@0 <= 62 AND 62 <= CounterID_max@1 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_max@4 AS Int32) AS Date32) >= 2013-07-01 AND EventDate_null_count@5 != row_count@3 AND CAST(CAST(EventDate_min@6 AS Int32) AS Date32) <= 2013-07-31 AND IsRefresh_null_count@9 != row_count@3 AND IsRefresh_min@7 <= 0 AND 0 <= IsRefresh_max@8 AND (TraficSourceID_null_count@12 != row_count@3 AND TraficSourceID_min@10 <= -1 AND -1 <= TraficSourceID_max@11 OR TraficSourceID_null_count@12 != row_count@3 AND TraficSourceID_min@10 <= 6 AND 6 <= TraficSourceID_max@11) AND RefererHash_null_count@15 != row_count@3 AND RefererHash_min@13 <= 3594120000172545465 AND 3594120000172545465 <= RefererHash_max@14, required_guarantees=[CounterID in (62), IsRefresh in (0), RefererHash in (3594120000172545465), TraficSourceID in (-1, 6)] diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index 3d0b50b6a61e9..0a30118493ca6 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -155,10 +155,9 @@ logical_plan 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id, info] physical_plan -01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Disable Join dynamic filter pushdown statement ok @@ -178,10 +177,9 @@ logical_plan 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id, info] physical_plan -01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Re-enable for next tests statement ok @@ -203,10 +201,9 @@ logical_plan 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id, info] physical_plan -01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +01)HashJoinExec: mode=CollectLeft, join_type=Right, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # LEFT JOIN correctness: all left rows appear, unmatched right rows produce NULLs query ITT @@ -236,10 +233,9 @@ logical_plan 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id, info] physical_plan -01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # RIGHT JOIN correctness: all right rows appear, unmatched left rows produce NULLs query ITT @@ -468,10 +464,9 @@ logical_plan 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id, info] physical_plan -01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Enable TopK, disable Join statement ok @@ -494,10 +489,9 @@ logical_plan 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id, info] physical_plan -01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Test 4: Aggregate dynamic filter pushdown @@ -643,10 +637,9 @@ logical_plan 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id, info] physical_plan -01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet # Re-enable statement ok @@ -691,10 +684,9 @@ logical_plan 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id, info] physical_plan -01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Test 6: Regression test for issue #20213 - dynamic filter applied to wrong table # when subquery join has same column names on both sides. diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 630fdc4b1480f..46d01f39a920b 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -284,38 +284,30 @@ ON ---- physical_plan 01)┌───────────────────────────┐ -02)│ ProjectionExec │ +02)│ HashJoinExec │ 03)│ -------------------- │ -04)│ date_col: date_col │ -05)│ │ -06)│ string_col: │ -07)│ string_col │ -08)└─────────────┬─────────────┘ -09)┌─────────────┴─────────────┐ -10)│ HashJoinExec │ -11)│ -------------------- │ -12)│ filter: │ -13)│ CAST(int_col + int_col AS │ -14)│ Int64) % 2 = 0 ├──────────────┐ -15)│ │ │ -16)│ on: │ │ -17)│ (int_col = int_col) │ │ -18)└─────────────┬─────────────┘ │ -19)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -20)│ DataSourceExec ││ RepartitionExec │ -21)│ -------------------- ││ -------------------- │ -22)│ files: 1 ││ partition_count(in->out): │ -23)│ format: parquet ││ 1 -> 4 │ -24)│ ││ │ -25)│ ││ partitioning_scheme: │ -26)│ ││ RoundRobinBatch(4) │ -27)└───────────────────────────┘└─────────────┬─────────────┘ -28)-----------------------------┌─────────────┴─────────────┐ -29)-----------------------------│ DataSourceExec │ -30)-----------------------------│ -------------------- │ -31)-----------------------------│ files: 1 │ -32)-----------------------------│ format: csv │ -33)-----------------------------└───────────────────────────┘ +04)│ filter: │ +05)│ CAST(int_col + int_col AS │ +06)│ Int64) % 2 = 0 ├──────────────┐ +07)│ │ │ +08)│ on: │ │ +09)│ (int_col = int_col) │ │ +10)└─────────────┬─────────────┘ │ +11)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +12)│ DataSourceExec ││ RepartitionExec │ +13)│ -------------------- ││ -------------------- │ +14)│ files: 1 ││ partition_count(in->out): │ +15)│ format: parquet ││ 1 -> 4 │ +16)│ ││ │ +17)│ ││ partitioning_scheme: │ +18)│ ││ RoundRobinBatch(4) │ +19)└───────────────────────────┘└─────────────┬─────────────┘ +20)-----------------------------┌─────────────┴─────────────┐ +21)-----------------------------│ DataSourceExec │ +22)-----------------------------│ -------------------- │ +23)-----------------------------│ files: 1 │ +24)-----------------------------│ format: csv │ +25)-----------------------------└───────────────────────────┘ # 3 Joins query TT @@ -329,49 +321,33 @@ FROM ---- physical_plan 01)┌───────────────────────────┐ -02)│ ProjectionExec │ +02)│ HashJoinExec │ 03)│ -------------------- │ -04)│ date_col: date_col │ -05)│ │ -06)│ string_col: │ -07)│ string_col │ -08)└─────────────┬─────────────┘ -09)┌─────────────┴─────────────┐ -10)│ HashJoinExec │ -11)│ -------------------- │ -12)│ on: ├──────────────┐ -13)│ (int_col = int_col) │ │ -14)└─────────────┬─────────────┘ │ -15)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -16)│ DataSourceExec ││ ProjectionExec │ -17)│ -------------------- ││ -------------------- │ -18)│ bytes: 512 ││ date_col: date_col │ -19)│ format: memory ││ int_col: int_col │ -20)│ rows: 1 ││ │ -21)│ ││ string_col: │ -22)│ ││ string_col │ -23)└───────────────────────────┘└─────────────┬─────────────┘ -24)-----------------------------┌─────────────┴─────────────┐ -25)-----------------------------│ HashJoinExec │ -26)-----------------------------│ -------------------- │ -27)-----------------------------│ on: ├──────────────┐ -28)-----------------------------│ (int_col = int_col) │ │ -29)-----------------------------└─────────────┬─────────────┘ │ -30)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -31)-----------------------------│ DataSourceExec ││ RepartitionExec │ -32)-----------------------------│ -------------------- ││ -------------------- │ -33)-----------------------------│ files: 1 ││ partition_count(in->out): │ -34)-----------------------------│ format: parquet ││ 1 -> 4 │ -35)-----------------------------│ ││ │ -36)-----------------------------│ predicate: ││ partitioning_scheme: │ -37)-----------------------------│ DynamicFilter [ empty ] ││ RoundRobinBatch(4) │ -38)-----------------------------└───────────────────────────┘└─────────────┬─────────────┘ -39)----------------------------------------------------------┌─────────────┴─────────────┐ -40)----------------------------------------------------------│ DataSourceExec │ -41)----------------------------------------------------------│ -------------------- │ -42)----------------------------------------------------------│ files: 1 │ -43)----------------------------------------------------------│ format: csv │ -44)----------------------------------------------------------└───────────────────────────┘ +04)│ on: ├──────────────┐ +05)│ (int_col = int_col) │ │ +06)└─────────────┬─────────────┘ │ +07)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +08)│ DataSourceExec ││ HashJoinExec │ +09)│ -------------------- ││ -------------------- │ +10)│ bytes: 512 ││ on: ├──────────────┐ +11)│ format: memory ││ (int_col = int_col) │ │ +12)│ rows: 1 ││ │ │ +13)└───────────────────────────┘└─────────────┬─────────────┘ │ +14)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +15)-----------------------------│ DataSourceExec ││ RepartitionExec │ +16)-----------------------------│ -------------------- ││ -------------------- │ +17)-----------------------------│ files: 1 ││ partition_count(in->out): │ +18)-----------------------------│ format: parquet ││ 1 -> 4 │ +19)-----------------------------│ ││ │ +20)-----------------------------│ predicate: ││ partitioning_scheme: │ +21)-----------------------------│ DynamicFilter [ empty ] ││ RoundRobinBatch(4) │ +22)-----------------------------└───────────────────────────┘└─────────────┬─────────────┘ +23)----------------------------------------------------------┌─────────────┴─────────────┐ +24)----------------------------------------------------------│ DataSourceExec │ +25)----------------------------------------------------------│ -------------------- │ +26)----------------------------------------------------------│ files: 1 │ +27)----------------------------------------------------------│ format: csv │ +28)----------------------------------------------------------└───────────────────────────┘ # Long Filter (demonstrate what happens with wrapping) query TT @@ -1075,40 +1051,28 @@ explain select * from table1 inner join table2 on table1.int_col = table2.int_co ---- physical_plan 01)┌───────────────────────────┐ -02)│ ProjectionExec │ +02)│ HashJoinExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col │ -06)│ │ -07)│ date_col: date_col │ -08)│ int_col: int_col │ -09)│ │ -10)│ string_col: │ -11)│ string_col │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ HashJoinExec │ -15)│ -------------------- │ -16)│ on: │ -17)│ (int_col = int_col), ├──────────────┐ -18)│ (string_col = │ │ -19)│ string_col) │ │ -20)└─────────────┬─────────────┘ │ -21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -22)│ DataSourceExec ││ RepartitionExec │ -23)│ -------------------- ││ -------------------- │ -24)│ files: 1 ││ partition_count(in->out): │ -25)│ format: parquet ││ 1 -> 4 │ -26)│ ││ │ -27)│ ││ partitioning_scheme: │ -28)│ ││ RoundRobinBatch(4) │ -29)└───────────────────────────┘└─────────────┬─────────────┘ -30)-----------------------------┌─────────────┴─────────────┐ -31)-----------------------------│ DataSourceExec │ -32)-----------------------------│ -------------------- │ -33)-----------------------------│ files: 1 │ -34)-----------------------------│ format: csv │ -35)-----------------------------└───────────────────────────┘ +04)│ on: │ +05)│ (int_col = int_col), ├──────────────┐ +06)│ (string_col = │ │ +07)│ string_col) │ │ +08)└─────────────┬─────────────┘ │ +09)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +10)│ DataSourceExec ││ RepartitionExec │ +11)│ -------------------- ││ -------------------- │ +12)│ files: 1 ││ partition_count(in->out): │ +13)│ format: parquet ││ 1 -> 4 │ +14)│ ││ │ +15)│ ││ partitioning_scheme: │ +16)│ ││ RoundRobinBatch(4) │ +17)└───────────────────────────┘└─────────────┬─────────────┘ +18)-----------------------------┌─────────────┴─────────────┐ +19)-----------------------------│ DataSourceExec │ +20)-----------------------------│ -------------------- │ +21)-----------------------------│ files: 1 │ +22)-----------------------------│ format: csv │ +23)-----------------------------└───────────────────────────┘ # Query with outer hash join. query TT @@ -1116,42 +1080,30 @@ explain select * from table1 left outer join table2 on table1.int_col = table2.i ---- physical_plan 01)┌───────────────────────────┐ -02)│ ProjectionExec │ +02)│ HashJoinExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col │ -06)│ │ -07)│ date_col: date_col │ -08)│ int_col: int_col │ -09)│ │ -10)│ string_col: │ -11)│ string_col │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ HashJoinExec │ -15)│ -------------------- │ -16)│ join_type: Right │ -17)│ │ -18)│ on: ├──────────────┐ -19)│ (int_col = int_col), │ │ -20)│ (string_col = │ │ -21)│ string_col) │ │ -22)└─────────────┬─────────────┘ │ -23)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -24)│ DataSourceExec ││ RepartitionExec │ -25)│ -------------------- ││ -------------------- │ -26)│ files: 1 ││ partition_count(in->out): │ -27)│ format: parquet ││ 1 -> 4 │ -28)│ ││ │ -29)│ ││ partitioning_scheme: │ -30)│ ││ RoundRobinBatch(4) │ -31)└───────────────────────────┘└─────────────┬─────────────┘ -32)-----------------------------┌─────────────┴─────────────┐ -33)-----------------------------│ DataSourceExec │ -34)-----------------------------│ -------------------- │ -35)-----------------------------│ files: 1 │ -36)-----------------------------│ format: csv │ -37)-----------------------------└───────────────────────────┘ +04)│ join_type: Right │ +05)│ │ +06)│ on: ├──────────────┐ +07)│ (int_col = int_col), │ │ +08)│ (string_col = │ │ +09)│ string_col) │ │ +10)└─────────────┬─────────────┘ │ +11)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +12)│ DataSourceExec ││ RepartitionExec │ +13)│ -------------------- ││ -------------------- │ +14)│ files: 1 ││ partition_count(in->out): │ +15)│ format: parquet ││ 1 -> 4 │ +16)│ ││ │ +17)│ ││ partitioning_scheme: │ +18)│ ││ RoundRobinBatch(4) │ +19)└───────────────────────────┘└─────────────┬─────────────┘ +20)-----------------------------┌─────────────┴─────────────┐ +21)-----------------------------│ DataSourceExec │ +22)-----------------------------│ -------------------- │ +23)-----------------------------│ files: 1 │ +24)-----------------------------│ format: csv │ +25)-----------------------------└───────────────────────────┘ # Query with nested loop join. query TT diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 366326479dab1..59db63ba420e9 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2025,10 +2025,9 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[last_value(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] -09)----------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)] -10)------------------DataSourceExec: partitions=1, partition_sizes=[3] -11)------------------DataSourceExec: partitions=1, partition_sizes=[3] +08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)], projection=[col0@2, col1@3, col2@4, col0@0, col1@1] +09)----------------DataSourceExec: partitions=1, partition_sizes=[3] +10)----------------DataSourceExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is DataSourceExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2946,10 +2945,9 @@ physical_plan 01)SortExec: expr=[sn@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, last_value(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] 03)----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[last_value(e.amount) ORDER BY [e.sn ASC NULLS LAST]] -04)------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----------DataSourceExec: partitions=1, partition_sizes=[1] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[zip_code@4, country@5, sn@6, ts@7, currency@8, sn@0, amount@3] +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--------DataSourceExec: partitions=1, partition_sizes=[1] query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -4081,14 +4079,13 @@ logical_plan 09)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[sum(CAST(multiple_ordered_table_with_pk.d AS Int64))]] 10)----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -01)ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] -03)----ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] -04)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true -06)----ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] -07)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -08)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, c@3, sum1@2, sum1@5] +02)--ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] +03)----AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true +05)--ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] +06)----AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +07)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], constraints=[PrimaryKey([3])], file_type=csv, has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index c0a838c97d552..b9d163d877596 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -1367,18 +1367,16 @@ logical_plan 06)----TableScan: f projection=[a] 07)--TableScan: s projection=[b] physical_plan -01)ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@3 as a, b@0 as b] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[b@0, col0@2, col1@3, a@4] -03)----ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)----ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] -06)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] -07)--------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] -08)----------DataSourceExec: partitions=1, partition_sizes=[1] -09)--------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] -10)----------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 -11)------------FilterExec: y@1 = x@0 -12)--------------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[col0@2, col1@3, a@4, b@0] +02)--ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[col0@2, col1@3, a@0] +05)----ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] +06)------DataSourceExec: partitions=1, partition_sizes=[1] +07)----ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] +08)------RepartitionExec: partitioning=RoundRobinBatch(16), input_partitions=1 +09)--------FilterExec: y@1 = x@0 +10)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok drop table pairs; @@ -1421,15 +1419,14 @@ logical_plan 05)------Aggregate: groupBy=[[t1.v0]], aggr=[[sum(t1.v1)]] 06)--------TableScan: t1 projection=[v0, v1] physical_plan -01)ProjectionExec: expr=[v0@1 as v0, v1@2 as v1, sum(t1.v1)@0 as sum(t1.v1)] -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(v0@1, v0@0)], projection=[sum(t1.v1)@0, v0@2, v1@3] -03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[sum(t1.v1)@1 as sum(t1.v1), v0@0 as v0] -05)--------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)] -06)----------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] -08)--------------DataSourceExec: partitions=1, partition_sizes=[1] -09)----DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Right, on=[(v0@1, v0@0)], projection=[v0@2, v1@3, sum(t1.v1)@0] +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[sum(t1.v1)@1 as sum(t1.v1), v0@0 as v0] +04)------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)] +05)--------RepartitionExec: partitioning=Hash([v0@0], 4), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[v0@0 as v0], aggr=[sum(t1.v1)] +07)------------DataSourceExec: partitions=1, partition_sizes=[1] +08)--DataSourceExec: partitions=1, partition_sizes=[1] query III SELECT * diff --git a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt index 2bab89c99eae0..1b6f2e4c86385 100644 --- a/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt +++ b/datafusion/sqllogictest/test_files/join_is_not_distinct_from.slt @@ -140,8 +140,8 @@ logical_plan 04)------TableScan: t1 projection=[id, val] 05)----TableScan: t2 projection=[id, val] physical_plan -01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], projection=[id@0, val@1, id@3, val@4], NullsEqual: true +01)ProjectionExec: expr=[id@0 as t1_id, id@1 as t2_id, val@2 as val, val@3 as val] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], projection=[id@0, id@3, val@1, val@4], NullsEqual: true 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -169,8 +169,8 @@ logical_plan 03)----TableScan: t1 projection=[id, val] 04)----TableScan: t2 projection=[id, val] physical_plan -01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], filter=CAST(val@0 AS Int64) % 3 IS DISTINCT FROM CAST(val@1 AS Int64) % 3, projection=[id@0, val@1, id@3, val@4], NullsEqual: true +01)ProjectionExec: expr=[id@0 as t1_id, id@1 as t2_id, val@2 as val, val@3 as val] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1.val + Int64(1)@2, t2.val + Int64(1)@2)], filter=CAST(val@0 AS Int64) % 3 IS DISTINCT FROM CAST(val@1 AS Int64) % 3, projection=[id@0, id@3, val@1, val@4], NullsEqual: true 03)----ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t1.val + Int64(1)] 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)----ProjectionExec: expr=[id@0 as id, val@1 as val, CAST(val@1 AS Int64) + 1 as t2.val + Int64(1)] @@ -229,8 +229,8 @@ logical_plan 05)------TableScan: t2 projection=[id, val] 06)----TableScan: t0 projection=[val] physical_plan -01)ProjectionExec: expr=[id@0 as t1_id, id@2 as t2_id, val@1 as val, val@3 as val] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@0, val@1)], projection=[id@1, val@2, id@3, val@4], NullsEqual: true +01)ProjectionExec: expr=[id@0 as t1_id, id@1 as t2_id, val@2 as val, val@3 as val] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@0, val@1)], projection=[id@1, id@3, val@2, val@4], NullsEqual: true 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(val@1, val@1)], NullsEqual: true diff --git a/datafusion/sqllogictest/test_files/join_limit_pushdown.slt b/datafusion/sqllogictest/test_files/join_limit_pushdown.slt index 24d48e5661ad1..ea9c971065411 100644 --- a/datafusion/sqllogictest/test_files/join_limit_pushdown.slt +++ b/datafusion/sqllogictest/test_files/join_limit_pushdown.slt @@ -74,10 +74,9 @@ logical_plan 04)----Limit: skip=0, fetch=3 05)------TableScan: t2 projection=[x], fetch=3 physical_plan -01)ProjectionExec: expr=[a@1 as a, x@0 as x] -02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(x@0, a@0)], fetch=3 -03)----DataSourceExec: partitions=1, partition_sizes=[1], fetch=3 -04)----DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(x@0, a@0)], projection=[a@1, x@0], fetch=3 +02)--DataSourceExec: partitions=1, partition_sizes=[1], fetch=3 +03)--DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT t1.a, t2.x FROM t1 RIGHT JOIN t2 ON t1.a = t2.x LIMIT 3; @@ -221,12 +220,11 @@ logical_plan 05)------TableScan: t2 projection=[x] 06)----TableScan: t3 projection=[p] physical_plan -01)ProjectionExec: expr=[a@1 as a, x@2 as x, p@0 as p] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p@0, x@1)], fetch=2 -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, x@0)] -05)------DataSourceExec: partitions=1, partition_sizes=[1] -06)------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(p@0, x@1)], projection=[a@1, x@2, p@0], fetch=2 +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, x@0)] +04)----DataSourceExec: partitions=1, partition_sizes=[1] +05)----DataSourceExec: partitions=1, partition_sizes=[1] query III SELECT t1.a, t2.x, t3.p diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 0c14958f1a87c..1ed3fc89b2642 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1536,15 +1536,14 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] -03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t1_id@2, t2_id@0, t1_name@3] +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +07)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)------DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1561,15 +1560,14 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] -03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t1_id@2, t2_id@0, t1_name@3] +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +07)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)------DataSourceExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1589,12 +1587,11 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t1_id@1, t2_id@0, t1_name@2] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1] # don't reorder join based on stats, and use the order in the query @@ -1612,14 +1609,13 @@ INNER JOIN join_t2 ON join_t1.t1_id + cast(11 as INT UNSIGNED) = join_t2.t2_id ---- physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, t2_id@2 as t2_id, t1_name@1 as t1_name] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)], projection=[t1_id@0, t1_name@1, t2_id@3] -03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + UInt32(11)@2, t2_id@0)], projection=[t1_id@0, t2_id@3, t1_name@1] +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.join_reordering = true; @@ -1643,12 +1639,11 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t1_id@1, t2_id@0, t1_name@2] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------DataSourceExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1668,14 +1663,13 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] -03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t1_id@2, t2_id@0, t1_name@3] +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----DataSourceExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1693,14 +1687,13 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] -03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)------DataSourceExec: partitions=1, partition_sizes=[1] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t1_id@2, t2_id@0, t1_name@3] +02)--CoalescePartitionsExec +03)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----DataSourceExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -2063,15 +2056,14 @@ logical_plan 05)----Filter: join_t2.t2_int > UInt32(1) 06)------TableScan: join_t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id] -02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 -03)----CoalescePartitionsExec -04)------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: partitions=1, partition_sizes=[1] -07)----FilterExec: t1_id@0 > 10 -08)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------DataSourceExec: partitions=1, partition_sizes=[1] +01)NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1, projection=[t1_id@1, t2_id@0] +02)--CoalescePartitionsExec +03)----FilterExec: t2_int@1 > 1, projection=[t2_id@0] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------DataSourceExec: partitions=1, partition_sizes=[1] +06)--FilterExec: t1_id@0 > 10 +07)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------DataSourceExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -3634,15 +3626,14 @@ logical_plan 09)----Projection: Int64(1) AS e, Int64(3) AS f 10)------EmptyRelation: rows=1 physical_plan -01)ProjectionExec: expr=[c@2 as c, d@3 as d, e@0 as e, f@1 as f] -02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] -03)----ProjectionExec: expr=[1 as e, 3 as f] -04)------PlaceholderRowExec -05)----UnionExec -06)------ProjectionExec: expr=[1 as c, 2 as d] -07)--------PlaceholderRowExec -08)------ProjectionExec: expr=[1 as c, 3 as d] -09)--------PlaceholderRowExec +01)HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)], projection=[c@2, d@3, e@0, f@1] +02)--ProjectionExec: expr=[1 as e, 3 as f] +03)----PlaceholderRowExec +04)--UnionExec +05)----ProjectionExec: expr=[1 as c, 2 as d] +06)------PlaceholderRowExec +07)----ProjectionExec: expr=[1 as c, 3 as d] +08)------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3676,15 +3667,14 @@ logical_plan 09)----Projection: Int64(1) AS e, Int64(3) AS f 10)------EmptyRelation: rows=1 physical_plan -01)ProjectionExec: expr=[c@2 as c, d@3 as d, e@0 as e, f@1 as f] -02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)] -03)----ProjectionExec: expr=[1 as e, 3 as f] -04)------PlaceholderRowExec -05)----UnionExec -06)------ProjectionExec: expr=[1 as c, 2 as d] -07)--------PlaceholderRowExec -08)------ProjectionExec: expr=[1 as c, 3 as d] -09)--------PlaceholderRowExec +01)HashJoinExec: mode=CollectLeft, join_type=Full, on=[(e@0, c@0)], projection=[c@2, d@3, e@0, f@1] +02)--ProjectionExec: expr=[1 as e, 3 as f] +03)----PlaceholderRowExec +04)--UnionExec +05)----ProjectionExec: expr=[1 as c, 2 as d] +06)------PlaceholderRowExec +07)----ProjectionExec: expr=[1 as c, 3 as d] +08)------PlaceholderRowExec query IIII rowsort SELECT * FROM ( @@ -3888,11 +3878,10 @@ logical_plan 05)----Sort: right_table_no_nulls.b ASC NULLS LAST, fetch=10 06)------TableScan: right_table_no_nulls projection=[a, b] physical_plan -01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] -02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -03)----SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[2] -05)----DataSourceExec: partitions=1, partition_sizes=[2] +01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)], projection=[a@2, b@3, a@0, b@1] +02)--SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] +03)----DataSourceExec: partitions=1, partition_sizes=[2] +04)--DataSourceExec: partitions=1, partition_sizes=[2] @@ -3945,10 +3934,9 @@ logical_plan 04)--SubqueryAlias: rhs 05)----TableScan: right_table_no_nulls projection=[a, b] physical_plan -01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] -02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -03)----DataSourceExec: partitions=1, partition_sizes=[2] -04)----DataSourceExec: partitions=1, partition_sizes=[2] +01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)], projection=[a@2, b@3, a@0, b@1] +02)--DataSourceExec: partitions=1, partition_sizes=[2] +03)--DataSourceExec: partitions=1, partition_sizes=[2] # Null build indices: @@ -4004,11 +3992,10 @@ logical_plan 05)----Sort: right_table_no_nulls.b ASC NULLS LAST, fetch=10 06)------TableScan: right_table_no_nulls projection=[a, b] physical_plan -01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] -02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] -03)----SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] -04)------DataSourceExec: partitions=1, partition_sizes=[2] -05)----DataSourceExec: partitions=1, partition_sizes=[2] +01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)], projection=[a@2, b@3, a@0, b@1] +02)--SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] +03)----DataSourceExec: partitions=1, partition_sizes=[2] +04)--DataSourceExec: partitions=1, partition_sizes=[2] # Test CROSS JOIN LATERAL syntax (planning) @@ -4908,10 +4895,9 @@ FROM t1 JOIN t2 ON k1 = k2 ---- physical_plan -01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k2@0, k1@0)], projection=[k1@2, v1@3, k2@0, v2@1] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -4928,10 +4914,9 @@ FROM t1 LEFT JOIN t2 ON k1 = k2 ---- physical_plan -01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=Right, on=[(k2@0, k1@0)], projection=[k1@2, v1@3, k2@0, v2@1] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -4948,10 +4933,9 @@ FROM t1 RIGHT JOIN t2 ON k1 = k2 ---- physical_plan -01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] -02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(k2@0, k1@0)], projection=[k1@2, v1@3, k2@0, v2@1] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) @@ -5044,10 +5028,9 @@ FROM t1 FULL JOIN t2 ON k1 = k2 ---- physical_plan -01)ProjectionExec: expr=[k1@2 as k1, v1@3 as v1, k2@0 as k2, v2@1 as v2] -02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----DataSourceExec: partitions=1, partition_sizes=[10000] +01)HashJoinExec: mode=CollectLeft, join_type=Full, on=[(k2@0, k1@0)], projection=[k1@2, v1@3, k2@0, v2@1] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[10000] query IIII SELECT sum(k1), sum(v1), sum(k2), sum(v2) diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 8bc2ca465e9b6..dd1f69b286a91 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -755,16 +755,15 @@ logical_plan 10)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan 01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[sum(lineitem.l_extendedprice), avg(lineitem.l_discount), count(DISTINCT partsupp.ps_suppkey)] -02)--ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey, ps_suppkey@0 as ps_suppkey] -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, p_partkey@2)], projection=[ps_suppkey@1, l_extendedprice@2, l_discount@3, p_partkey@4] -04)------DataSourceExec: partitions=1, partition_sizes=[1] -05)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] -06)--------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 -07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true -08)--------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -09)----------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -10)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(ps_partkey@0, p_partkey@2)], projection=[l_extendedprice@2, l_discount@3, p_partkey@4, ps_suppkey@1] +03)----DataSourceExec: partitions=1, partition_sizes=[1] +04)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +05)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=1 +06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], file_type=csv, has_header=true +07)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +08)--------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +09)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], file_type=csv, has_header=true # Simplification of a binary operator with a NULL value diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 7c24a6bd2a5db..48d99625eabd1 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -362,13 +362,12 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[max(d.env), max(d.service), count(Int64(1)), sum(f.value)], ordering_mode=Sorted 04)------RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3, preserve_order=true, sort_exprs=f_dkey@0 ASC NULLS LAST 05)--------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[max(d.env), max(d.service), count(Int64(1)), sum(f.value)], ordering_mode=Sorted -06)----------ProjectionExec: expr=[value@2 as value, f_dkey@3 as f_dkey, env@0 as env, service@1 as service] -07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[env@1, service@2, value@3, f_dkey@4] -08)--------------CoalescePartitionsExec -09)----------------FilterExec: service@2 = log -10)------------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet]]}, projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -12)--------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[value@3, f_dkey@4, env@1, service@2] +07)------------CoalescePartitionsExec +08)--------------FilterExec: service@2 = log +09)----------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +10)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet]]}, projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +11)------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results without optimization query TTTIR rowsort @@ -414,13 +413,12 @@ physical_plan 01)SortPreservingMergeExec: [f_dkey@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[f_dkey@0 as f_dkey, max(d.env)@1 as max(d.env), max(d.service)@2 as max(d.service), count(Int64(1))@3 as count(*), sum(f.value)@4 as sum(f.value)] 03)----AggregateExec: mode=SinglePartitioned, gby=[f_dkey@1 as f_dkey], aggr=[max(d.env), max(d.service), count(Int64(1)), sum(f.value)], ordering_mode=Sorted -04)------ProjectionExec: expr=[value@2 as value, f_dkey@3 as f_dkey, env@0 as env, service@1 as service] -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[env@1, service@2, value@3, f_dkey@4] -06)----------CoalescePartitionsExec -07)------------FilterExec: service@2 = log -08)--------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet]]}, projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -10)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[value@3, f_dkey@4, env@1, service@2] +05)--------CoalescePartitionsExec +06)----------FilterExec: service@2 = log +07)------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet]]}, projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +09)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] query TTTIR rowsort SELECT f.f_dkey, MAX(d.env), MAX(d.service), count(*), sum(f.value) @@ -641,12 +639,11 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)] 02)--RepartitionExec: partitioning=Hash([f_dkey@0, env@1], 3), input_partitions=3 03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)] -04)------ProjectionExec: expr=[value@1 as value, f_dkey@2 as f_dkey, env@0 as env] -05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[env@0, value@2, f_dkey@3] -06)----------RepartitionExec: partitioning=Hash([d_dkey@1], 3), input_partitions=3 -07)------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet -08)----------RepartitionExec: partitioning=Hash([f_dkey@1], 3), input_partitions=3 -09)------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet, predicate=DynamicFilter [ empty ] +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[value@2, f_dkey@3, env@0] +05)--------RepartitionExec: partitioning=Hash([d_dkey@1], 3), input_partitions=3 +06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet +07)--------RepartitionExec: partitioning=Hash([f_dkey@1], 3), input_partitions=3 +08)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet, predicate=DynamicFilter [ empty ] query TTR rowsort SELECT f.f_dkey, d.env, sum(f.value) @@ -687,10 +684,9 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, env@1 as env], aggr=[sum(f.value)] 02)--RepartitionExec: partitioning=Hash([f_dkey@0, env@1], 3), input_partitions=3 03)----AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey, env@2 as env], aggr=[sum(f.value)] -04)------ProjectionExec: expr=[value@1 as value, f_dkey@2 as f_dkey, env@0 as env] -05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[env@0, value@2, f_dkey@3] -06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet -07)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[value@2, f_dkey@3, env@0] +05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet +06)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet query TTR rowsort SELECT f.f_dkey, d.env, sum(f.value) diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index 777f1e00ed312..a40c1b8c7e246 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -1533,8 +1533,8 @@ logical_plan 05)----Projection: get_field(join_right.s, Utf8("role")) AS __datafusion_extracted_2, join_right.id 06)------TableScan: join_right projection=[id, s] physical_plan -01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[label], __datafusion_extracted_2@2 as join_right.s[role]] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@1, id@1)], projection=[__datafusion_extracted_1@0, id@1, __datafusion_extracted_2@2] +01)ProjectionExec: expr=[id@0 as id, __datafusion_extracted_1@1 as simple_struct.s[label], __datafusion_extracted_2@2 as join_right.s[role]] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@1, id@1)], projection=[id@1, __datafusion_extracted_1@0, __datafusion_extracted_2@2] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, id], file_type=parquet 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, role) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] @@ -1603,8 +1603,8 @@ logical_plan 07)--------Projection: get_field(join_right.s, Utf8("level")) AS __datafusion_extracted_1, join_right.id, get_field(join_right.s, Utf8("level")) AS __datafusion_extracted_3 08)----------TableScan: join_right projection=[id, s], partial_filters=[get_field(join_right.s, Utf8("level")) > Int64(5)] physical_plan -01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_2@0 as simple_struct.s[value], __datafusion_extracted_3@2 as join_right.s[level]] -02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@1, id@0)], projection=[__datafusion_extracted_2@0, id@1, __datafusion_extracted_3@3] +01)ProjectionExec: expr=[id@0 as id, __datafusion_extracted_2@1 as simple_struct.s[value], __datafusion_extracted_3@2 as join_right.s[level]] +02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@1, id@0)], projection=[id@1, __datafusion_extracted_2@0, __datafusion_extracted_3@3] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet 04)----FilterExec: __datafusion_extracted_1@0 > 5, projection=[id@1, __datafusion_extracted_3@2] 05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_1, id, get_field(s@1, level) as __datafusion_extracted_3], file_type=parquet, predicate=get_field(s@1, level) > 5 AND DynamicFilter [ empty ] @@ -1896,8 +1896,8 @@ logical_plan 07)------Projection: get_field(simple_struct.s, Utf8("value")) AS __datafusion_extracted_1, get_field(simple_struct.s, Utf8("value")) AS __datafusion_extracted_3, simple_struct.id 08)--------TableScan: simple_struct projection=[id, s] physical_plan -01)ProjectionExec: expr=[__datafusion_extracted_3@1 as s.s[value], __datafusion_extracted_4@0 as j.s[role]] -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@2, id@2)], filter=__datafusion_extracted_1@1 > __datafusion_extracted_2@0, projection=[__datafusion_extracted_4@1, __datafusion_extracted_3@4] +01)ProjectionExec: expr=[__datafusion_extracted_3@0 as s.s[value], __datafusion_extracted_4@1 as j.s[role]] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@2, id@2)], filter=__datafusion_extracted_1@1 > __datafusion_extracted_2@0, projection=[__datafusion_extracted_3@4, __datafusion_extracted_4@1] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, get_field(s@1, role) as __datafusion_extracted_4, id], file_type=parquet 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_3, id], file_type=parquet, predicate=DynamicFilter [ empty ] diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index fd49fd900488a..dbf31dec5e118 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -376,12 +376,11 @@ physical_plan 08)--------------AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@2 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) 09)----------------RepartitionExec: partitioning=Hash([f_dkey@0, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1, env@2], 3), input_partitions=3, preserve_order=true, sort_exprs=f_dkey@0 ASC NULLS LAST, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 ASC NULLS LAST 10)------------------AggregateExec: mode=Partial, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) -11)--------------------ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] -12)----------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] -13)------------------------CoalescePartitionsExec -14)--------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] -15)----------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -16)------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +11)--------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[f_dkey@4, env@0, timestamp@2, value@3] +12)----------------------CoalescePartitionsExec +13)------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] +14)--------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +15)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results without subset satisfaction query TPR rowsort @@ -472,12 +471,11 @@ physical_plan 06)----------AggregateExec: mode=Partial, gby=[env@1 as env, time_bin@0 as time_bin], aggr=[avg(a.max_bin_value)] 07)------------ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp)@1 as time_bin, env@2 as env, max(j.value)@3 as max_bin_value] 08)--------------AggregateExec: mode=SinglePartitioned, gby=[f_dkey@0 as f_dkey, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }, timestamp@2) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 30000000000 }"),j.timestamp), env@1 as env], aggr=[max(j.value)], ordering_mode=PartiallySorted([0, 1]) -09)----------------ProjectionExec: expr=[f_dkey@3 as f_dkey, env@0 as env, timestamp@1 as timestamp, value@2 as value] -10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[env@0, timestamp@2, value@3, f_dkey@4] -11)--------------------CoalescePartitionsExec -12)----------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] -13)------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -14)--------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +09)----------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@1, f_dkey@2)], projection=[f_dkey@4, env@0, timestamp@2, value@3] +10)------------------CoalescePartitionsExec +11)--------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] +12)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +13)------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results match with subset satisfaction query TPR rowsort diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 7f88199b3c0ef..5d54de6dbce2d 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -200,8 +200,8 @@ logical_plan 06)--------Aggregate: groupBy=[[t2.t2_id]], aggr=[[sum(CAST(t2.t2_int AS Int64))]] 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +01)ProjectionExec: expr=[t1_id@0 as t1_id, sum(t2.t2_int)@1 as t2_sum] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] 05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] @@ -233,8 +233,8 @@ logical_plan 06)--------Aggregate: groupBy=[[t2.t2_id]], aggr=[[sum(CAST(t2.t2_int AS Float64)) AS sum(t2.t2_int * Float64(1))]] 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int * Float64(1)) + Int64(1)@0, t1_id@2] +01)ProjectionExec: expr=[t1_id@0 as t1_id, sum(t2.t2_int * Float64(1)) + Int64(1)@1 as t2_sum] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int * Float64(1)) + Int64(1)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int * Float64(1))@1 + 1 as sum(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] 05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] @@ -266,8 +266,8 @@ logical_plan 06)--------Aggregate: groupBy=[[t2.t2_id]], aggr=[[sum(CAST(t2.t2_int AS Int64))]] 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +01)ProjectionExec: expr=[t1_id@0 as t1_id, sum(t2.t2_int)@1 as t2_sum] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] 05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] @@ -300,18 +300,17 @@ logical_plan 07)----------Aggregate: groupBy=[[t2.t2_id]], aggr=[[sum(CAST(t2.t2_int AS Int64))]] 08)------------TableScan: t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +01)ProjectionExec: expr=[t1_id@0 as t1_id, sum(t2.t2_int)@1 as t2_sum] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[t1_id@2, sum(t2.t2_int)@0] 03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] -05)--------FilterExec: sum(t2.t2_int)@1 < 3 -06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------DataSourceExec: partitions=1, partition_sizes=[2] -11)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)------DataSourceExec: partitions=1, partition_sizes=[2] +04)------FilterExec: sum(t2.t2_int)@1 < 3, projection=[sum(t2.t2_int)@1, t2_id@0] +05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------DataSourceExec: partitions=1, partition_sizes=[2] +10)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------DataSourceExec: partitions=1, partition_sizes=[2] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index 84237e4393377..7e3617b1d597c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -53,6 +53,6 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] 05)--------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(Int64(1))] -07)------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] -08)--------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] +07)------------ProjectionExec: expr=[l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as __common_expr_1, l_quantity@2 as l_quantity, l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] +08)--------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_extendedprice@1, l_discount@2, l_quantity@0, l_tax@3, l_returnflag@4, l_linestatus@5] 09)----------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index d5ff6724402ad..b1a15388270b3 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -101,48 +101,46 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], fetch=10 02)--SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] -04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] -05)--------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] -07)------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@10, n_regionkey@11] -09)----------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 -10)------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@3 as s_name, s_address@4 as s_address, s_nationkey@5 as s_nationkey, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@2 as ps_supplycost] -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_supplycost@3, s_name@5, s_address@6, s_nationkey@7, s_phone@8, s_acctbal@9, s_comment@10] -12)----------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] -14)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -15)----------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] -16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false -18)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -19)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false -20)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -21)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=csv, has_header=false -22)----------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -23)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false -24)------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -25)--------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -26)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false -28)--------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -29)----------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -30)------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -31)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -32)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -33)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -34)--------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -35)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -36)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -37)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -38)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -39)------------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false -40)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 -41)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false -42)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 -43)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false -44)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -45)----------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -46)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)--------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[s_acctbal@5, s_name@2, n_name@8, p_partkey@0, p_mfgr@1, s_address@3, s_phone@4, s_comment@6] +04)------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] +06)----------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@10, n_regionkey@11] +08)--------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@5, s_address@6, s_nationkey@7, s_phone@8, s_acctbal@9, s_comment@10, ps_supplycost@3] +10)------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] +12)----------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +13)------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] +14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], file_type=csv, has_header=false +16)----------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +17)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false +18)------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +19)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], file_type=csv, has_header=false +20)--------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +21)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], file_type=csv, has_header=false +22)----------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +23)------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +24)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false +26)------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +27)--------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +28)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +29)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +30)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +31)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +32)------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +33)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +34)----------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +35)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +36)--------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +37)----------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], file_type=csv, has_header=false +38)--------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=1 +39)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], file_type=csv, has_header=false +40)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=1 +41)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], file_type=csv, has_header=false +42)------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +43)--------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +44)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index b4e70993396e6..590a737703847 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -90,8 +90,8 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] 05)--------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] -07)------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] +07)------------ProjectionExec: expr=[n_name@0 as supp_nation, n_name@1 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@3 * (Some(1),20,0 - l_discount@4) as volume] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[n_name@4, n_name@6, l_shipdate@2, l_extendedprice@0, l_discount@1] 09)----------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)], projection=[l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@4, n_name@6] 11)--------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 12f19d43d40e7..7f160ce072c4d 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -96,8 +96,8 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] 05)--------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[sum(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), sum(all_nations.volume)] -07)------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] +07)------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@0) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as volume, n_name@3 as nation] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[o_orderdate@2, l_extendedprice@0, l_discount@1, n_name@4] 09)----------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] 11)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 4ec434c90368f..961fde94bb1ee 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -81,8 +81,8 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 05)--------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] -07)------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] +07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@3 * (Some(1),20,0 - l_discount@4) - ps_supplycost@5 * l_quantity@2 as amount] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4] 09)----------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 diff --git a/testing b/testing index 7df2b70baf4f0..0d60ccae40d0e 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 7df2b70baf4f081ebf8e0c6bd22745cf3cbfd824 +Subproject commit 0d60ccae40d0e8f2d22c15fafb01c5d4be8c63a6 From 15bbe4a13d7dd66777842593e73db2cb3f4efb74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 3 Apr 2026 11:08:23 +0200 Subject: [PATCH 2/7] Add insta tests --- datafusion/core/tests/dataframe/mod.rs | 98 +++++++++---------- .../physical_optimizer/projection_pushdown.rs | 4 +- 2 files changed, 50 insertions(+), 52 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 80bbde1f6ba14..1c661744e0867 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -3429,31 +3429,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+----------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+----------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | ProjectionExec: expr=[a@2 as a, b@3 as b, count(*)@0 as count(*), __always_true@1 as __always_true] | - | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[count(*)@0, __always_true@2, a@3, b@4] | - | | CoalescePartitionsExec | - | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+----------------------------------------------------------------------------------------------------------------------------+ + +---------------+--------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+--------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(Int64(1)) AS count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | + | | CoalescePartitionsExec | + | | ProjectionExec: expr=[count(Int64(1))@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+--------------------------------------------------------------------------------------------------------------------------+ " ); @@ -3485,31 +3484,30 @@ async fn test_count_wildcard_on_where_scalar_subquery() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+----------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+----------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | Projection: t1.a, t1.b | - | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | - | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | - | | Left Join: t1.a = __scalar_sq_1.a | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __scalar_sq_1 | - | | Projection: count(*), t2.a, Boolean(true) AS __always_true | - | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[a] | - | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | ProjectionExec: expr=[a@2 as a, b@3 as b, count(*)@0 as count(*), __always_true@1 as __always_true] | - | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[count(*)@0, __always_true@2, a@3, b@4] | - | | CoalescePartitionsExec | - | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+----------------------------------------------------------------------------------------------------------------------------+ + +---------------+--------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+--------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | Projection: t1.a, t1.b | + | | Filter: CASE WHEN __scalar_sq_1.__always_true IS NULL THEN Int64(0) ELSE __scalar_sq_1.count(*) END > Int64(0) | + | | Projection: t1.a, t1.b, __scalar_sq_1.count(*), __scalar_sq_1.__always_true | + | | Left Join: t1.a = __scalar_sq_1.a | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __scalar_sq_1 | + | | Projection: count(*), t2.a, Boolean(true) AS __always_true | + | | Aggregate: groupBy=[[t2.a]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[a] | + | physical_plan | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@1, a@0)], projection=[a@3, b@4, count(*)@0, __always_true@2] | + | | CoalescePartitionsExec | + | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+--------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index d3448062d1062..9155999c2837f 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -1313,8 +1313,8 @@ fn test_hash_join_after_projection() -> Result<()> { assert_snapshot!( actual, @r" - ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@3 as c_from_right] - HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7] + ProjectionExec: expr=[c@0 as c_from_left, b@1 as b_from_left, a@2 as a_from_left, c@3 as c_from_right] + HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[c@2, b@1, a@0, c@7] 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 " From 68646e871def9ac65b6af429e3d262a6c8636ba7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 3 Apr 2026 11:33:03 +0200 Subject: [PATCH 3/7] Use expression tree traversal order instead of sorting for complex expression column collection Co-Authored-By: Claude Opus 4.6 (1M context) --- datafusion/physical-plan/src/projection.rs | 41 +++++++++---------- .../sqllogictest/test_files/lateral_join.slt | 4 +- .../test_files/tpch/plans/q9.slt.part | 4 +- 3 files changed, 24 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 75e9a568a79fe..cd8e110efbcef 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -51,7 +51,6 @@ use datafusion_execution::TaskContext; use datafusion_expr::ExpressionPlacement; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::projection::Projector; -use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr_common::physical_expr::{PhysicalExprRef, fmt_sql}; use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexRequirement, PhysicalSortExpr, @@ -1082,14 +1081,11 @@ fn try_unifying_projections( /// Collect all column indices from the given projection expressions. fn collect_column_indices(exprs: &[ProjectionExpr]) -> Vec { // Collect column indices in a deterministic order that preserves the - // projection's column ordering when possible. For simple Column - // expressions, we use the column index directly (preserving the - // projection's desired output order). For complex expressions with - // multiple column references, we sort indices for determinism since - // collect_columns returns a HashSet with non-deterministic iteration. + // projection's column ordering. For simple Column expressions, we use + // the column index directly. For complex expressions, we walk the + // expression tree to collect column references in traversal order. // This allows the embedded projection to match the desired output - // column order for simple column reorderings, avoiding a residual - // ProjectionExec. + // column order, avoiding a residual ProjectionExec. let mut seen = std::collections::HashSet::new(); let mut indices = Vec::new(); for proj_expr in exprs { @@ -1099,18 +1095,20 @@ fn collect_column_indices(exprs: &[ProjectionExpr]) -> Vec { indices.push(col.index()); } } else { - // Complex expression: collect all referenced columns in sorted - // order for determinism. - let mut expr_indices: Vec = collect_columns(&proj_expr.expr) - .into_iter() - .map(|c| c.index()) - .collect(); - expr_indices.sort(); - for idx in expr_indices { - if seen.insert(idx) { - indices.push(idx); - } - } + // Complex expression: collect all referenced columns in + // expression tree traversal order (deterministic) to preserve + // the natural ordering of column references. + proj_expr + .expr + .apply(|expr| { + if let Some(col) = expr.as_any().downcast_ref::() { + if seen.insert(col.index()) { + indices.push(col.index()); + } + } + Ok(TreeNodeRecursion::Continue) + }) + .expect("closure always returns OK"); } } indices @@ -1226,7 +1224,8 @@ mod tests { expr, alias: "b-(1+a)".to_string(), }]); - assert_eq!(column_indices, vec![1, 7]); + // Tree traversal order: b@7 is visited before a@1 + assert_eq!(column_indices, vec![7, 1]); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/lateral_join.slt b/datafusion/sqllogictest/test_files/lateral_join.slt index ddc2fe586029a..ec34d62a439ae 100644 --- a/datafusion/sqllogictest/test_files/lateral_join.slt +++ b/datafusion/sqllogictest/test_files/lateral_join.slt @@ -527,8 +527,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[id@0 as id, CASE WHEN __always_true@2 IS NULL THEN 0 ELSE cnt@1 END as cnt] -04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, cnt@1, __always_true@3] +03)----ProjectionExec: expr=[id@0 as id, CASE WHEN __always_true@1 IS NULL THEN 0 ELSE cnt@2 END as cnt] +04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, __always_true@3, cnt@1] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------ProjectionExec: expr=[count(Int64(1))@1 as cnt, t1_id@0 as t1_id, true as __always_true] 07)----------AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 961fde94bb1ee..7a973490be479 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -81,8 +81,8 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 05)--------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] -07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@3 * (Some(1),20,0 - l_discount@4) - ps_supplycost@5 * l_quantity@2 as amount] -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4] +07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@5 as amount] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_extendedprice@1, l_discount@2, ps_supplycost@4, l_quantity@0] 09)----------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 From 587064af1b9fe4eb8fa623c1b2c4811219da43ce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 3 Apr 2026 11:34:19 +0200 Subject: [PATCH 4/7] Revert testing submodule to master Co-Authored-By: Claude Opus 4.6 (1M context) --- testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing b/testing index 0d60ccae40d0e..7df2b70baf4f0 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 0d60ccae40d0e8f2d22c15fafb01c5d4be8c63a6 +Subproject commit 7df2b70baf4f081ebf8e0c6bd22745cf3cbfd824 From 8037db542ac92afaa56ca31e5d8c9a403517e4e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 3 Apr 2026 11:45:47 +0200 Subject: [PATCH 5/7] Revert "Use expression tree traversal order instead of sorting for complex expression column collection" This reverts commit 68646e871def9ac65b6af429e3d262a6c8636ba7. --- datafusion/physical-plan/src/projection.rs | 41 ++++++++++--------- .../sqllogictest/test_files/lateral_join.slt | 4 +- .../test_files/tpch/plans/q9.slt.part | 4 +- 3 files changed, 25 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index cd8e110efbcef..75e9a568a79fe 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -51,6 +51,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::ExpressionPlacement; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::projection::Projector; +use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr_common::physical_expr::{PhysicalExprRef, fmt_sql}; use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexRequirement, PhysicalSortExpr, @@ -1081,11 +1082,14 @@ fn try_unifying_projections( /// Collect all column indices from the given projection expressions. fn collect_column_indices(exprs: &[ProjectionExpr]) -> Vec { // Collect column indices in a deterministic order that preserves the - // projection's column ordering. For simple Column expressions, we use - // the column index directly. For complex expressions, we walk the - // expression tree to collect column references in traversal order. + // projection's column ordering when possible. For simple Column + // expressions, we use the column index directly (preserving the + // projection's desired output order). For complex expressions with + // multiple column references, we sort indices for determinism since + // collect_columns returns a HashSet with non-deterministic iteration. // This allows the embedded projection to match the desired output - // column order, avoiding a residual ProjectionExec. + // column order for simple column reorderings, avoiding a residual + // ProjectionExec. let mut seen = std::collections::HashSet::new(); let mut indices = Vec::new(); for proj_expr in exprs { @@ -1095,20 +1099,18 @@ fn collect_column_indices(exprs: &[ProjectionExpr]) -> Vec { indices.push(col.index()); } } else { - // Complex expression: collect all referenced columns in - // expression tree traversal order (deterministic) to preserve - // the natural ordering of column references. - proj_expr - .expr - .apply(|expr| { - if let Some(col) = expr.as_any().downcast_ref::() { - if seen.insert(col.index()) { - indices.push(col.index()); - } - } - Ok(TreeNodeRecursion::Continue) - }) - .expect("closure always returns OK"); + // Complex expression: collect all referenced columns in sorted + // order for determinism. + let mut expr_indices: Vec = collect_columns(&proj_expr.expr) + .into_iter() + .map(|c| c.index()) + .collect(); + expr_indices.sort(); + for idx in expr_indices { + if seen.insert(idx) { + indices.push(idx); + } + } } } indices @@ -1224,8 +1226,7 @@ mod tests { expr, alias: "b-(1+a)".to_string(), }]); - // Tree traversal order: b@7 is visited before a@1 - assert_eq!(column_indices, vec![7, 1]); + assert_eq!(column_indices, vec![1, 7]); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/lateral_join.slt b/datafusion/sqllogictest/test_files/lateral_join.slt index ec34d62a439ae..ddc2fe586029a 100644 --- a/datafusion/sqllogictest/test_files/lateral_join.slt +++ b/datafusion/sqllogictest/test_files/lateral_join.slt @@ -527,8 +527,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[id@0 as id, CASE WHEN __always_true@1 IS NULL THEN 0 ELSE cnt@2 END as cnt] -04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, __always_true@3, cnt@1] +03)----ProjectionExec: expr=[id@0 as id, CASE WHEN __always_true@2 IS NULL THEN 0 ELSE cnt@1 END as cnt] +04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, cnt@1, __always_true@3] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------ProjectionExec: expr=[count(Int64(1))@1 as cnt, t1_id@0 as t1_id, true as __always_true] 07)----------AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 7a973490be479..961fde94bb1ee 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -81,8 +81,8 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 05)--------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] -07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@5 as amount] -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_extendedprice@1, l_discount@2, ps_supplycost@4, l_quantity@0] +07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@3 * (Some(1),20,0 - l_discount@4) - ps_supplycost@5 * l_quantity@2 as amount] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4] 09)----------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 From 54ac038381d2916967a8fc65a81e1eba5afd205d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 3 Apr 2026 11:47:16 +0200 Subject: [PATCH 6/7] Reapply "Use expression tree traversal order instead of sorting for complex expression column collection" This reverts commit 8037db542ac92afaa56ca31e5d8c9a403517e4e7. --- datafusion/physical-plan/src/projection.rs | 41 +++++++++---------- .../sqllogictest/test_files/lateral_join.slt | 4 +- .../test_files/tpch/plans/q9.slt.part | 4 +- 3 files changed, 24 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 75e9a568a79fe..cd8e110efbcef 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -51,7 +51,6 @@ use datafusion_execution::TaskContext; use datafusion_expr::ExpressionPlacement; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::projection::Projector; -use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr_common::physical_expr::{PhysicalExprRef, fmt_sql}; use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexRequirement, PhysicalSortExpr, @@ -1082,14 +1081,11 @@ fn try_unifying_projections( /// Collect all column indices from the given projection expressions. fn collect_column_indices(exprs: &[ProjectionExpr]) -> Vec { // Collect column indices in a deterministic order that preserves the - // projection's column ordering when possible. For simple Column - // expressions, we use the column index directly (preserving the - // projection's desired output order). For complex expressions with - // multiple column references, we sort indices for determinism since - // collect_columns returns a HashSet with non-deterministic iteration. + // projection's column ordering. For simple Column expressions, we use + // the column index directly. For complex expressions, we walk the + // expression tree to collect column references in traversal order. // This allows the embedded projection to match the desired output - // column order for simple column reorderings, avoiding a residual - // ProjectionExec. + // column order, avoiding a residual ProjectionExec. let mut seen = std::collections::HashSet::new(); let mut indices = Vec::new(); for proj_expr in exprs { @@ -1099,18 +1095,20 @@ fn collect_column_indices(exprs: &[ProjectionExpr]) -> Vec { indices.push(col.index()); } } else { - // Complex expression: collect all referenced columns in sorted - // order for determinism. - let mut expr_indices: Vec = collect_columns(&proj_expr.expr) - .into_iter() - .map(|c| c.index()) - .collect(); - expr_indices.sort(); - for idx in expr_indices { - if seen.insert(idx) { - indices.push(idx); - } - } + // Complex expression: collect all referenced columns in + // expression tree traversal order (deterministic) to preserve + // the natural ordering of column references. + proj_expr + .expr + .apply(|expr| { + if let Some(col) = expr.as_any().downcast_ref::() { + if seen.insert(col.index()) { + indices.push(col.index()); + } + } + Ok(TreeNodeRecursion::Continue) + }) + .expect("closure always returns OK"); } } indices @@ -1226,7 +1224,8 @@ mod tests { expr, alias: "b-(1+a)".to_string(), }]); - assert_eq!(column_indices, vec![1, 7]); + // Tree traversal order: b@7 is visited before a@1 + assert_eq!(column_indices, vec![7, 1]); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/lateral_join.slt b/datafusion/sqllogictest/test_files/lateral_join.slt index ddc2fe586029a..ec34d62a439ae 100644 --- a/datafusion/sqllogictest/test_files/lateral_join.slt +++ b/datafusion/sqllogictest/test_files/lateral_join.slt @@ -527,8 +527,8 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[id@0 as id, CASE WHEN __always_true@2 IS NULL THEN 0 ELSE cnt@1 END as cnt] -04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, cnt@1, __always_true@3] +03)----ProjectionExec: expr=[id@0 as id, CASE WHEN __always_true@1 IS NULL THEN 0 ELSE cnt@2 END as cnt] +04)------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, __always_true@3, cnt@1] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------ProjectionExec: expr=[count(Int64(1))@1 as cnt, t1_id@0 as t1_id, true as __always_true] 07)----------AggregateExec: mode=FinalPartitioned, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 961fde94bb1ee..7a973490be479 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -81,8 +81,8 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 05)--------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] -07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@3 * (Some(1),20,0 - l_discount@4) - ps_supplycost@5 * l_quantity@2 as amount] -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4] +07)------------ProjectionExec: expr=[n_name@0 as nation, date_part(YEAR, o_orderdate@1) as o_year, l_extendedprice@2 * (Some(1),20,0 - l_discount@3) - ps_supplycost@4 * l_quantity@5 as amount] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[n_name@7, o_orderdate@5, l_extendedprice@1, l_discount@2, ps_supplycost@4, l_quantity@0] 09)----------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 From c9a91d528f535dd7e635468210d9479fc52ef276 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 3 Apr 2026 11:59:17 +0200 Subject: [PATCH 7/7] Fix clippy collapsible_if warning Co-Authored-By: Claude Opus 4.6 (1M context) --- datafusion/physical-plan/src/projection.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index cd8e110efbcef..fc3c4f72e7d57 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -1101,10 +1101,10 @@ fn collect_column_indices(exprs: &[ProjectionExpr]) -> Vec { proj_expr .expr .apply(|expr| { - if let Some(col) = expr.as_any().downcast_ref::() { - if seen.insert(col.index()) { - indices.push(col.index()); - } + if let Some(col) = expr.as_any().downcast_ref::() + && seen.insert(col.index()) + { + indices.push(col.index()); } Ok(TreeNodeRecursion::Continue) })