From ab81469f043e892b5988faed382b36738c112677 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 7 Aug 2025 22:33:37 +0800 Subject: [PATCH 01/72] ``` feat: enhance predicate handling for join optimization - Retain inferred predicates that cannot be pushed through joins as join filters for dynamic filter pushdown. - Update join filter assertions in tests to reflect new logic. - Add tests for dynamic filter pushdown scenarios, including: - Left join with a filter on the preserved side. - Right join with a filter on the preserved side. - Handling filters that do not restrict nulls. ``` --- datafusion/optimizer/src/push_down_filter.rs | 96 +++++++++++++++++++- 1 file changed, 92 insertions(+), 4 deletions(-) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 35ec7d074d5f..e07dc767fb76 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -453,12 +453,17 @@ fn push_down_all_join( } } - // For infer predicates, if they can not push through join, just drop them + // For inferred predicates, if they can not be pushed to either side of the join + // (for example because the corresponding input is not preserved by the join + // type), retain them as join filters so they can still participate in + // dynamic filter pushdown. for predicate in inferred_join_predicates { if left_preserved && checker.is_left_only(&predicate) { left_push.push(predicate); } else if right_preserved && checker.is_right_only(&predicate) { right_push.push(predicate); + } else { + join_conditions.push(predicate); } } @@ -2689,7 +2694,7 @@ mod tests { assert_optimized_plan_equal!( plan, @r" - Left Join: Using test.a = test2.a + Left Join: Using test.a = test2.a Filter: test2.a <= Int64(1) TableScan: test, full_filters=[test.a <= Int64(1)] Projection: test2.a TableScan: test2 @@ -2730,7 +2735,7 @@ mod tests { assert_optimized_plan_equal!( plan, @r" - Right Join: Using test.a = test2.a + Right Join: Using test.a = test2.a Filter: test.a <= Int64(1) TableScan: test Projection: test2.a TableScan: test2, full_filters=[test2.a <= Int64(1)] @@ -2909,7 +2914,7 @@ mod tests { assert_optimized_plan_equal!( plan, @r" - Left Join: test.a = test2.a Filter: test.a > UInt32(1) AND test.b < test2.b + Left Join: test.a = test2.a Filter: test2.a > UInt32(1) AND test.a > UInt32(1) AND test.b < test2.b Projection: test.a, test.b, test.c TableScan: test Projection: test2.a, test2.b, test2.c @@ -2964,6 +2969,89 @@ mod tests { ) } + /// A filter on the preserved(left) side of a left join should be + /// converted into a join filter on the right side via dynamic filter pushdown + #[test] + fn left_join_dynamic_filter_pushdown() -> Result<()> { + let left = test_table_scan()?; + let right = test_table_scan_with_name("test2")?; + + let filter = col("test.a").gt(lit(1u32)); + let plan = LogicalPlanBuilder::from(left) + .join( + right, + JoinType::Left, + (vec![Column::from_name("a")], vec![Column::from_name("a")]), + None, + )? + .filter(filter)? + .build()?; + + assert_optimized_plan_equal!( + plan, + @r" + Left Join: test.a = test2.a Filter: test2.a > UInt32(1) + TableScan: test, full_filters=[test.a > UInt32(1)] + TableScan: test2 + " + ) + } + + /// A filter on the preserved(right) side of a right join should be + /// converted into a join filter on the left side via dynamic filter pushdown + #[test] + fn right_join_dynamic_filter_pushdown() -> Result<()> { + let left = test_table_scan()?; + let right = test_table_scan_with_name("test2")?; + + let filter = col("test2.a").gt(lit(1u32)); + let plan = LogicalPlanBuilder::from(left) + .join( + right, + JoinType::Right, + (vec![Column::from_name("a")], vec![Column::from_name("a")]), + None, + )? + .filter(filter)? + .build()?; + + assert_optimized_plan_equal!( + plan, + @r" + Right Join: test.a = test2.a Filter: test.a > UInt32(1) + TableScan: test + TableScan: test2, full_filters=[test2.a > UInt32(1)] + " + ) + } + + /// Filters that do not restrict nulls should not generate dynamic filters + #[test] + fn left_join_dynamic_filter_pushdown_with_nulls() -> Result<()> { + let left = test_table_scan()?; + let right = test_table_scan_with_name("test2")?; + + let filter = col("test.a").is_null(); + let plan = LogicalPlanBuilder::from(left) + .join( + right, + JoinType::Left, + (vec![Column::from_name("a")], vec![Column::from_name("a")]), + None, + )? + .filter(filter)? + .build()?; + + assert_optimized_plan_equal!( + plan, + @r" + Left Join: test.a = test2.a + TableScan: test, full_filters=[test.a IS NULL] + TableScan: test2 + " + ) + } + /// single table predicate parts of ON condition should not be pushed #[test] fn full_join_on_with_filter() -> Result<()> { From 15c88973fdaedd30931e77f44f5a4e916f0dafe6 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 8 Aug 2025 21:54:32 +0800 Subject: [PATCH 02/72] Refactor join logic in test file to improve filter application and execution plan clarity --- .../join_disable_repartition_joins.slt | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index a1efc1317b4a..c214778eea03 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -89,20 +89,25 @@ logical_plan 02)--Projection: t2.a AS a2, t2.b 03)----RightSemi Join: t1.d = t2.d, t1.c = t2.c 04)------SubqueryAlias: t1 -05)--------TableScan: annotated_data projection=[c, d] -06)------SubqueryAlias: t2 -07)--------Filter: annotated_data.d = Int32(3) -08)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] +05)--------Filter: annotated_data.d = Int32(3) +06)----------TableScan: annotated_data projection=[c, d], partial_filters=[annotated_data.d = Int32(3)] +07)------SubqueryAlias: t2 +08)--------Filter: annotated_data.d = Int32(3) +09)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] physical_plan 01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true -06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------FilterExec: d@3 = 3 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true +05)--------CoalescePartitionsExec +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------FilterExec: d@1 = 3 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true +10)--------CoalesceBatchesExec: target_batch_size=8192 +11)----------FilterExec: d@3 = 3 +12)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true # preserve_right_semi_join query II nosort From c960bdaa6058a4cb6680bdfae42c1bfb5c501a07 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 8 Aug 2025 23:40:49 +0800 Subject: [PATCH 03/72] Fix filter application order in RightAnti join execution plan --- datafusion/sqllogictest/test_files/joins.slt | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 5d68ed35b2a9..8ab67cecebbc 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -5127,10 +5127,12 @@ WHERE k1 < 0 physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] -03)----DataSourceExec: partitions=1, partition_sizes=[0] -04)----CoalesceBatchesExec: target_batch_size=3 -05)------FilterExec: k1@0 < 0 -06)--------DataSourceExec: partitions=1, partition_sizes=[10000] +03)----CoalesceBatchesExec: target_batch_size=3 +04)------FilterExec: k2@0 < 0 +05)--------DataSourceExec: partitions=1, partition_sizes=[0] +06)----CoalesceBatchesExec: target_batch_size=3 +07)------FilterExec: k1@0 < 0 +08)--------DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * From 1076e8b4f717b6ec410d4f62c4c27b67e713bcb5 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 12 Aug 2025 18:36:15 +0800 Subject: [PATCH 04/72] Enhance dynamic filter pushdown documentation for joins pushdown --- datafusion/common/src/config.rs | 17 +++++++++---- .../test_files/information_schema.slt | 2 +- dev/changelog/50.0.0.md | 24 +++++++++++++++++++ docs/source/library-user-guide/upgrading.md | 20 ++++++++++++++++ docs/source/user-guide/configs.md | 2 +- 5 files changed, 58 insertions(+), 7 deletions(-) create mode 100644 dev/changelog/50.0.0.md diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 939d13d9690e..fa7df7a5ff8c 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -725,11 +725,18 @@ config_namespace! { /// during aggregations, if possible pub enable_topk_aggregation: bool, default = true - /// When set to true attempts to push down dynamic filters generated by operators into the file scan phase. - /// For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer - /// will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. - /// This means that if we already have 10 timestamps in the year 2025 - /// any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. + /// When set to true attempts to push down dynamic filters generated by operators + /// into the file scan phase. For example, for a query such as + /// `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt + /// to push down the current top 10 timestamps that the TopK operator references + /// into the file scans. This means that if we already have 10 timestamps in the + /// year 2025 any files that only have timestamps in the year 2024 can be skipped / + /// pruned at various stages in the scan. Dynamic filters are also produced by + /// left, right, semi, and anti joins, allowing DataFusion to prune the probe side + /// during execution. Full joins are not supported. For example, + /// `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` + /// will only read `fact` rows whose `id` values match `dim` rows where + /// `region = 'US'`. pub enable_dynamic_filter_pushdown: bool, default = true /// When set to true, the optimizer will insert filters before a join between diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index c87abb972ea6..68cb6f2eeca7 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -399,7 +399,7 @@ datafusion.format.types_info false Show types in visual representation batches datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. -datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attempts to push down dynamic filters generated by operators into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. +datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attempts to push down dynamic filters generated by operators into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. Dynamic filters are also produced by left, right, semi, and anti joins, allowing DataFusion to prune the probe side during execution. Full joins are not supported. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible datafusion.optimizer.expand_views_at_output false When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. diff --git a/dev/changelog/50.0.0.md b/dev/changelog/50.0.0.md new file mode 100644 index 000000000000..4ac23495d9cd --- /dev/null +++ b/dev/changelog/50.0.0.md @@ -0,0 +1,24 @@ + + +# Apache DataFusion 50.0.0 Changelog + +This release is under active development. + +- Enable dynamic filter pushdown for left, right, semi, and anti joins [#16445](https://github.com/apache/datafusion/pull/16445) (adriangb) diff --git a/docs/source/library-user-guide/upgrading.md b/docs/source/library-user-guide/upgrading.md index 09e905f7ff79..d42dc4066a78 100644 --- a/docs/source/library-user-guide/upgrading.md +++ b/docs/source/library-user-guide/upgrading.md @@ -38,6 +38,26 @@ In order to enable single value optimizations and be consistent with other user defined function APIs, the `AsyncScalarUDFImpl::invoke_async_with_args` method now returns a `ColumnarValue` instead of a `ArrayRef`. +### Dynamic filter pushdown for joins + +Dynamic filter pushdown now applies to left, right, semi and anti joins, +allowing DataFusion to prune the probe side as join keys are discovered at +runtime. Full joins are not supported. This behavior is controlled by the +`datafusion.optimizer.enable_dynamic_filter_pushdown` configuration option (on by +default). + +For example: + +```sql +SELECT * +FROM fact LEFT JOIN dim + ON fact.id = dim.id +WHERE dim.region = 'US'; +``` + +As rows from `dim` with `region = 'US'` are processed, a dynamic filter is +generated that skips `fact` partitions without matching `id` values. + To upgrade, change the return type of your implementation ```rust diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 9895c4b6654a..8c57502221fd 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -129,7 +129,7 @@ The following configuration settings are available: | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. | +| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. Dynamic filters are also produced by left, right, semi, and anti joins, allowing DataFusion to prune the probe side during execution. Full joins are not supported. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. | | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | | datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | From fb1c85cd3c0fb067e78a460dd09977c1317ededb Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 12 Aug 2025 18:48:41 +0800 Subject: [PATCH 05/72] Implement dynamic filter pushdown for various join types in HashJoinExec tests --- .../physical_optimizer/filter_pushdown/mod.rs | 166 +++++++++++++++++- 1 file changed, 165 insertions(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 1639960fdeac..978b8575b218 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -32,7 +32,7 @@ use datafusion::{ prelude::{ParquetReadOptions, SessionConfig, SessionContext}, scalar::ScalarValue, }; -use datafusion_common::config::ConfigOptions; +use datafusion_common::{assert_contains, assert_not_contains, config::ConfigOptions, JoinType}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::ScalarUDF; use datafusion_functions::math::random::RandomFunc; @@ -48,6 +48,7 @@ use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, coalesce_batches::CoalesceBatchesExec, filter::FilterExec, + joins::{HashJoinExec, PartitionMode}, repartition::RepartitionExec, sorts::sort::SortExec, ExecutionPlan, @@ -1096,6 +1097,169 @@ async fn test_hashjoin_parent_filter_pushdown() { ); } +fn build_join_with_dynamic_filter( + join_type: JoinType, + left_support: bool, + right_support: bool, +) -> Arc { + let left_batches = vec![record_batch!( + ("a", Utf8, ["aa", "ab"]), + ("b", Utf8, ["ba", "bb"]), + ("c", Float64, [1.0, 2.0]) + ) + .unwrap()]; + let left_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, false), + Field::new("b", DataType::Utf8, false), + Field::new("c", DataType::Float64, false), + ])); + let left_scan = TestScanBuilder::new(Arc::clone(&left_schema)) + .with_support(left_support) + .with_batches(left_batches) + .build(); + + let right_batches = vec![record_batch!( + ("a", Utf8, ["aa", "ab", "ac", "ad"]), + ("b", Utf8, ["ba", "bb", "bc", "bd"]), + ("e", Float64, [1.0, 2.0, 3.0, 4.0]) + ) + .unwrap()]; + let right_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, false), + Field::new("b", DataType::Utf8, false), + Field::new("e", DataType::Float64, false), + ])); + let right_scan = TestScanBuilder::new(Arc::clone(&right_schema)) + .with_support(right_support) + .with_batches(right_batches) + .build(); + + let on = vec![ + (col("a", &left_schema).unwrap(), col("a", &right_schema).unwrap()), + (col("b", &left_schema).unwrap(), col("b", &right_schema).unwrap()), + ]; + + Arc::new( + HashJoinExec::try_new( + left_scan, + right_scan, + on, + None, + &join_type, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ) + .unwrap(), + ) as Arc +} + +#[tokio::test] +async fn test_hashjoin_left_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::Left, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_not_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate" + ); +} + +#[tokio::test] +async fn test_hashjoin_right_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::Right, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_not_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + ); +} + +#[tokio::test] +async fn test_hashjoin_left_semi_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::LeftSemi, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_not_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate" + ); +} + +#[tokio::test] +async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::LeftAnti, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_not_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate" + ); +} + +#[tokio::test] +async fn test_hashjoin_full_dynamic_filter_no_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::Full, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); +} + +#[tokio::test] +async fn test_hashjoin_dynamic_filter_pushdown_unsupported() { + let plan = build_join_with_dynamic_filter(JoinType::Left, true, false); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); +} + /// Integration test for dynamic filter pushdown with TopK. /// We use an integration test because there are complex interactions in the optimizer rules /// that the unit tests applying a single optimizer rule do not cover. From adf1207c2da854b792a119d5cf9a4709ccb2c0a4 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 12 Aug 2025 18:49:29 +0800 Subject: [PATCH 06/72] Implement dynamic filter handling for various join types in HashJoinExec --- .../physical-plan/src/joins/hash_join.rs | 141 +++++++++++++----- 1 file changed, 102 insertions(+), 39 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index d3999c6cd824..82a16922dd74 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -94,6 +94,53 @@ use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::{ready, Stream, StreamExt, TryStreamExt}; use parking_lot::Mutex; +// Helper functions copied from optimizer's push_down_filter module +// Determine which sides of a JOIN preserve rows for join output filters +fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { + match join_type { + JoinType::Inner => (true, true), + JoinType::Left => (true, false), + JoinType::Right => (false, true), + JoinType::Full => (false, false), + JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => (true, false), + JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => (false, true), + } +} + +// Determine which sides of a JOIN are preserved for ON-clause filters +fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { + match join_type { + JoinType::Inner => (true, true), + JoinType::Left => (false, true), + JoinType::Right => (true, false), + JoinType::Full => (false, false), + JoinType::LeftSemi | JoinType::RightSemi => (true, true), + JoinType::LeftAnti => (false, true), + JoinType::RightAnti => (true, false), + JoinType::LeftMark => (false, true), + JoinType::RightMark => (true, false), + } +} + +// Determine which side should receive the dynamic filter +fn dynamic_filter_side(join_type: JoinType) -> JoinSide { + let (left_preserved, right_preserved) = lr_is_preserved(join_type); + let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join_type); + match ( + left_preserved, + right_preserved, + on_left_preserved, + on_right_preserved, + ) { + // Filter left when right side is preserved but left is not + (false, true, true, _) => JoinSide::Left, + // Filter right when left side is preserved but right is not + (true, false, _, true) | (true, true, _, _) => JoinSide::Right, + // For full joins or unsupported cases, skip dynamic filtering + _ => JoinSide::None, + } +} + /// Hard-coded seed to ensure hash values from the hash join differ from `RepartitionExec`, avoiding collisions. const HASH_JOIN_SEED: RandomState = RandomState::with_seeds('J' as u64, 'O' as u64, 'I' as u64, 'N' as u64); @@ -413,7 +460,7 @@ impl HashJoinExec { projection.as_ref(), )?; - let dynamic_filter = Self::create_dynamic_filter(&on); + let dynamic_filter = Self::create_dynamic_filter(&on, *join_type); Ok(HashJoinExec { left, @@ -434,11 +481,20 @@ impl HashJoinExec { }) } - fn create_dynamic_filter(on: &JoinOn) -> Arc { - // Extract the right-side keys from the `on` clauses - let right_keys: Vec<_> = on.iter().map(|(_, r)| Arc::clone(r)).collect(); + fn create_dynamic_filter( + on: &JoinOn, + join_type: JoinType, + ) -> Arc { + // Determine which side of the join the dynamic filter should target + let filter_side = dynamic_filter_side(join_type); + // Extract the corresponding join keys from the `on` clauses + let keys: Vec<_> = match filter_side { + JoinSide::Left => on.iter().map(|(l, _)| Arc::clone(l)).collect(), + JoinSide::Right => on.iter().map(|(_, r)| Arc::clone(r)).collect(), + JoinSide::None => Vec::new(), + }; // Initialize with a placeholder expression (true) that will be updated when the hash table is built - Arc::new(DynamicFilterPhysicalExpr::new(right_keys, lit(true))) + Arc::new(DynamicFilterPhysicalExpr::new(keys, lit(true))) } /// left (build) side which gets hashed @@ -819,7 +875,7 @@ impl ExecutionPlan for HashJoinExec { column_indices: self.column_indices.clone(), null_equality: self.null_equality, cache: self.cache.clone(), - dynamic_filter: Self::create_dynamic_filter(&self.on), + dynamic_filter: Self::create_dynamic_filter(&self.on, self.join_type), })) } @@ -1017,35 +1073,54 @@ impl ExecutionPlan for HashJoinExec { parent_filters: Vec>, config: &ConfigOptions, ) -> Result { - // Other types of joins can support *some* filters, but restrictions are complex and error prone. - // For now we don't support them. - // See the logical optimizer rules for more details: datafusion/optimizer/src/push_down_filter.rs - // See https://github.com/apache/datafusion/issues/16973 for tracking. - if self.join_type != JoinType::Inner { - return Ok(FilterDescription::all_unsupported( + let (left_preserved, right_preserved) = lr_is_preserved(self.join_type); + + let unsupported: Vec<_> = parent_filters + .iter() + .map(|f| { + crate::filter_pushdown::PushedDownPredicate::unsupported(Arc::clone(f)) + }) + .collect(); + + let mut left_child = if left_preserved { + crate::filter_pushdown::ChildFilterDescription::from_child( &parent_filters, - &self.children(), - )); - } + self.left(), + )? + } else { + crate::filter_pushdown::ChildFilterDescription { + parent_filters: unsupported.clone(), + self_filters: vec![], + } + }; - // Get basic filter descriptions for both children - let left_child = crate::filter_pushdown::ChildFilterDescription::from_child( - &parent_filters, - self.left(), - )?; - let mut right_child = crate::filter_pushdown::ChildFilterDescription::from_child( - &parent_filters, - self.right(), - )?; + let mut right_child = if right_preserved { + crate::filter_pushdown::ChildFilterDescription::from_child( + &parent_filters, + self.right(), + )? + } else { + crate::filter_pushdown::ChildFilterDescription { + parent_filters: unsupported.clone(), + self_filters: vec![], + } + }; - // Add dynamic filters in Post phase if enabled if matches!(phase, FilterPushdownPhase::Post) && config.optimizer.enable_dynamic_filter_pushdown { - // Add actual dynamic filter to right side (probe side) + let target_side = dynamic_filter_side(self.join_type); let dynamic_filter = Arc::clone(&self.dynamic_filter) as Arc; - right_child = right_child.with_self_filter(dynamic_filter); + match target_side { + JoinSide::Left => { + left_child = left_child.with_self_filter(dynamic_filter); + } + JoinSide::Right => { + right_child = right_child.with_self_filter(dynamic_filter); + } + JoinSide::None => {} + } } Ok(FilterDescription::new() @@ -1059,18 +1134,6 @@ impl ExecutionPlan for HashJoinExec { child_pushdown_result: ChildPushdownResult, _config: &ConfigOptions, ) -> Result>> { - // Note: this check shouldn't be necessary because we already marked all parent filters as unsupported for - // non-inner joins in `gather_filters_for_pushdown`. - // However it's a cheap check and serves to inform future devs touching this function that they need to be really - // careful pushing down filters through non-inner joins. - if self.join_type != JoinType::Inner { - // Other types of joins can support *some* filters, but restrictions are complex and error prone. - // For now we don't support them. - // See the logical optimizer rules for more details: datafusion/optimizer/src/push_down_filter.rs - return Ok(FilterPushdownPropagation::all_unsupported( - child_pushdown_result, - )); - } Ok(FilterPushdownPropagation::if_any(child_pushdown_result)) } } From 081bed01241c1c5ca28413591ac467cb3780da69 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 12 Aug 2025 18:57:53 +0800 Subject: [PATCH 07/72] Refactor static filter pushdown test to include predicate in DataSourceExec output --- .../core/tests/physical_optimizer/filter_pushdown/mod.rs | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 978b8575b218..1ca940e597f3 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -417,10 +417,9 @@ async fn test_static_filter_pushdown_through_hash_join() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true output: Ok: - - FilterExec: a@0 = aa - - HashJoinExec: mode=Partitioned, join_type=Left, on=[(a@0, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true + - HashJoinExec: mode=Partitioned, join_type=Left, on=[(a@0, d@0)] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = aa + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true " ); } From 4675cbcb1c4dc50131655d182ae66f7d8bea6d78 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 12 Aug 2025 19:03:26 +0800 Subject: [PATCH 08/72] Add predicate information to explain tree output for parquet format --- datafusion/sqllogictest/test_files/explain_tree.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index f57c50506893..a0f90b857a9c 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1241,7 +1241,7 @@ physical_plan 31)│ -------------------- ││ -------------------- │ 32)│ files: 1 ││ partition_count(in->out): │ 33)│ format: parquet ││ 1 -> 4 │ -34)│ ││ │ +34)│ predicate: true ││ │ 35)│ ││ partitioning_scheme: │ 36)│ ││ RoundRobinBatch(4) │ 37)└───────────────────────────┘└─────────────┬─────────────┘ From 78b9f37f695d7a2f160fdf45c7fc84831f9d4ab0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Sat, 9 Aug 2025 20:20:51 +0800 Subject: [PATCH 09/72] Implement deduplication for join key pairs and partition expressions; add inferred predicate alias handling --- datafusion/core/src/physical_planner.rs | 73 ++++++- datafusion/expr/src/expr.rs | 9 +- datafusion/expr/src/utils.rs | 60 +++++- .../src/extract_equijoin_predicate.rs | 185 +++++++++++++++++- datafusion/optimizer/src/push_down_filter.rs | 54 ++++- .../src/enforce_distribution.rs | 12 +- 6 files changed, 377 insertions(+), 16 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 1021abc9e47b..427b423be9b8 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -18,7 +18,7 @@ //! Planner for [`LogicalPlan`] to [`ExecutionPlan`] use std::borrow::Cow; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; use crate::datasource::file_format::file_type_to_format; @@ -151,6 +151,46 @@ pub trait ExtensionPlanner { ) -> Result>>; } +fn expr_canonical_string(expr: &Arc) -> String { + format!("{}", expr) +} + +fn dedupe_join_key_pairs( + left: Vec>, + right: Vec>, +) -> (Vec>, Vec>) { + assert_eq!(left.len(), right.len()); + let mut seen = HashSet::new(); + let mut out_left = Vec::new(); + let mut out_right = Vec::new(); + for (l, r) in left.into_iter().zip(right.into_iter()) { + let l_s = expr_canonical_string(&l); + let r_s = expr_canonical_string(&r); + let key = if l_s <= r_s { + format!("{}={}", l_s, r_s) + } else { + format!("{}={}", r_s, l_s) + }; + if seen.insert(key) { + out_left.push(l); + out_right.push(r); + } + } + (out_left, out_right) +} + +fn dedupe_partition_exprs(exprs: Vec>) -> Vec> { + let mut seen = HashSet::new(); + let mut out = Vec::new(); + for e in exprs.into_iter() { + let key = expr_canonical_string(&e); + if seen.insert(key) { + out.push(e); + } + } + out +} + /// Default single node physical query planner that converts a /// `LogicalPlan` to an `ExecutionPlan` suitable for execution. /// @@ -858,6 +898,7 @@ impl DefaultPhysicalPlanner { ) }) .collect::>>()?; + let runtime_expr = dedupe_partition_exprs(runtime_expr); Partitioning::Hash(runtime_expr, *n) } LogicalPartitioning::DistributeBy(_) => { @@ -1088,6 +1129,10 @@ impl DefaultPhysicalPlanner { Ok((l, r)) }) .collect::>()?; + let (left_keys, right_keys): (Vec<_>, Vec<_>) = join_on.into_iter().unzip(); + let (left_keys, right_keys) = dedupe_join_key_pairs(left_keys, right_keys); + let join_on: join_utils::JoinOn = + left_keys.into_iter().zip(right_keys).collect(); let join_filter = match filter { Some(expr) => { @@ -2345,6 +2390,7 @@ impl<'a> OptimizationInvariantChecker<'a> { Ok(()) } + } impl<'n> TreeNodeVisitor<'n> for OptimizationInvariantChecker<'_> { @@ -3520,4 +3566,29 @@ digraph { Ok(()) } + + #[test] + fn test_dedupe_join_key_pairs() { + let l1: Arc = Arc::new(Column::new("a", 0)); + let r1: Arc = Arc::new(Column::new("b", 0)); + let l2: Arc = Arc::new(Column::new("b", 0)); + let r2: Arc = Arc::new(Column::new("a", 0)); + let (left, right) = dedupe_join_key_pairs(vec![l1.clone(), l2], vec![r1.clone(), r2]); + assert_eq!(left.len(), 1); + assert_eq!(format!("{}", left[0]), "a@0"); + assert_eq!(format!("{}", right[0]), "b@0"); + } + + #[test] + fn test_dedupe_partition_exprs() { + let exprs: Vec> = vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]; + let result = dedupe_partition_exprs(exprs); + assert_eq!(result.len(), 2); + assert_eq!(format!("{}", result[0]), "a@0"); + assert_eq!(format!("{}", result[1]), "b@1"); + } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index bb376b591696..17019a125681 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -38,6 +38,7 @@ use datafusion_common::tree_node::{ use datafusion_common::{ Column, DFSchema, HashMap, Result, ScalarValue, Spans, TableReference, }; +use crate::utils::INFERRED_PREDICATE_ALIAS; use datafusion_functions_window_common::field::WindowUDFFieldArgs; use sqlparser::ast::{ display_comma_separated, ExceptSelectItem, ExcludeSelectItem, IlikeSelectItem, @@ -3183,7 +3184,13 @@ pub const UNNEST_COLUMN_PREFIX: &str = "UNNEST"; impl Display for Expr { fn fmt(&self, f: &mut Formatter) -> fmt::Result { match self { - Expr::Alias(Alias { expr, name, .. }) => write!(f, "{expr} AS {name}"), + Expr::Alias(Alias { expr, name, .. }) => { + if name == INFERRED_PREDICATE_ALIAS { + write!(f, "{expr}") + } else { + write!(f, "{expr} AS {name}") + } + } Expr::Column(c) => write!(f, "{c}"), Expr::OuterReferenceColumn(_, c) => { write!(f, "{OUTER_REFERENCE_COLUMN_PREFIX}({c})") diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 80ad0f87846a..cbbe42db6f51 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -28,6 +28,15 @@ use crate::{ }; use datafusion_expr_common::signature::{Signature, TypeSignature}; +/// Alias used to mark inferred join predicates that should remain join filters +/// and not be converted into additional equijoin keys during optimization. +pub const INFERRED_PREDICATE_ALIAS: &str = "__datafusion_inferred_join_predicate"; + +/// Returns true if expr is an alias used to mark inferred join predicates. +pub fn is_inferred_alias(expr: &Expr) -> bool { + matches!(expr, Expr::Alias(Alias { name, .. }) if name == INFERRED_PREDICATE_ALIAS) +} + use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, @@ -946,7 +955,9 @@ fn split_conjunction_impl<'a>(expr: &'a Expr, mut exprs: Vec<&'a Expr>) -> Vec<& let exprs = split_conjunction_impl(left, exprs); split_conjunction_impl(right, exprs) } - Expr::Alias(Alias { expr, .. }) => split_conjunction_impl(expr, exprs), + Expr::Alias(alias) if !is_inferred_alias(expr) => { + split_conjunction_impl(&alias.expr, exprs) + } other => { exprs.push(other); exprs @@ -970,7 +981,11 @@ pub fn iter_conjunction(expr: &Expr) -> impl Iterator { stack.push(right); stack.push(left); } - Expr::Alias(Alias { expr, .. }) => stack.push(expr), + Expr::Alias(Alias { expr, name, .. }) + if name != INFERRED_PREDICATE_ALIAS => + { + stack.push(expr); + } other => return Some(other), } } @@ -994,7 +1009,11 @@ pub fn iter_conjunction_owned(expr: Expr) -> impl Iterator { stack.push(*right); stack.push(*left); } - Expr::Alias(Alias { expr, .. }) => stack.push(*expr), + Expr::Alias(Alias { expr, name, .. }) + if name != INFERRED_PREDICATE_ALIAS => + { + stack.push(*expr); + } other => return Some(other), } } @@ -1063,8 +1082,13 @@ fn split_binary_owned_impl( let exprs = split_binary_owned_impl(*left, operator, exprs); split_binary_owned_impl(*right, operator, exprs) } - Expr::Alias(Alias { expr, .. }) => { - split_binary_owned_impl(*expr, operator, exprs) + Expr::Alias(alias) => { + if is_inferred_alias(&Expr::Alias(alias.clone())) { + exprs.push(Expr::Alias(alias)); + exprs + } else { + split_binary_owned_impl(*alias.expr, operator, exprs) + } } other => { exprs.push(other); @@ -1090,7 +1114,9 @@ fn split_binary_impl<'a>( let exprs = split_binary_impl(left, operator, exprs); split_binary_impl(right, operator, exprs) } - Expr::Alias(Alias { expr, .. }) => split_binary_impl(expr, operator, exprs), + Expr::Alias(alias) if !is_inferred_alias(expr) => { + split_binary_impl(&alias.expr, operator, exprs) + } other => { exprs.push(other); exprs @@ -1648,6 +1674,17 @@ mod tests { assert_eq!(result, vec![&expr1, &expr2]); } + #[test] + fn test_split_conjunction_inferred_alias() { + let expr = col("a").eq(col("b")).alias(INFERRED_PREDICATE_ALIAS); + let result = split_conjunction(&expr); + assert_eq!(result.len(), 1); + match result[0] { + Expr::Alias(alias) => assert_eq!(alias.name, INFERRED_PREDICATE_ALIAS), + _ => panic!("expected alias"), + } + } + #[test] fn test_split_conjunction_or() { let expr = col("a").eq(lit(5)).or(col("b")); @@ -1705,6 +1742,17 @@ mod tests { ); } + #[test] + fn test_split_conjunction_owned_inferred_alias() { + let expr = col("a").eq(col("b")).alias(INFERRED_PREDICATE_ALIAS); + let result = split_conjunction_owned(expr); + assert_eq!(result.len(), 1); + match &result[0] { + Expr::Alias(alias) => assert_eq!(alias.name, INFERRED_PREDICATE_ALIAS), + _ => panic!("expected alias"), + } + } + #[test] fn test_conjunction_empty() { assert_eq!(conjunction(vec![]), None); diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index 55cf33ef4304..63515f1b0bc5 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -22,8 +22,9 @@ use datafusion_common::tree_node::Transformed; use datafusion_common::DFSchema; use datafusion_common::Result; use datafusion_expr::utils::split_conjunction_owned; -use datafusion_expr::utils::{can_hash, find_valid_equijoin_key_pair}; +use datafusion_expr::utils::{can_hash, find_valid_equijoin_key_pair, is_inferred_alias}; use datafusion_expr::{BinaryExpr, Expr, ExprSchemable, Join, LogicalPlan, Operator}; +use std::collections::{HashMap, HashSet}; // equijoin predicate type EquijoinPredicate = (Expr, Expr); @@ -81,14 +82,18 @@ impl OptimizerRule for ExtractEquijoinPredicate { let right_schema = right.schema(); let (equijoin_predicates, non_equijoin_expr) = split_eq_and_noneq_join_predicate(expr, left_schema, right_schema)?; + let has_new_keys = !equijoin_predicates.is_empty(); - if !equijoin_predicates.is_empty() { - on.extend(equijoin_predicates); + on.extend(equijoin_predicates); + on = dedupe_join_on(on); + let filter = residual_minus_on(non_equijoin_expr, &on); + + if has_new_keys { Ok(Transformed::yes(LogicalPlan::Join(Join { left, right, on, - filter: non_equijoin_expr, + filter, join_type, join_constraint, schema, @@ -99,7 +104,7 @@ impl OptimizerRule for ExtractEquijoinPredicate { left, right, on, - filter: non_equijoin_expr, + filter, join_type, join_constraint, schema, @@ -122,6 +127,10 @@ fn split_eq_and_noneq_join_predicate( let mut accum_join_keys: Vec<(Expr, Expr)> = vec![]; let mut accum_filters: Vec = vec![]; for expr in exprs { + if is_inferred_alias(&expr) { + accum_filters.push(expr); + continue; + } match expr { Expr::BinaryExpr(BinaryExpr { ref left, @@ -152,17 +161,151 @@ fn split_eq_and_noneq_join_predicate( Ok((accum_join_keys, result_filter)) } +#[derive(Default)] +struct UnionFind { + parent: HashMap, +} + +impl UnionFind { + fn find(&mut self, x: String) -> String { + let p = self.parent.get(&x).cloned().unwrap_or_else(|| x.clone()); + if p != x { + let r = self.find(p.clone()); + self.parent.insert(x, r.clone()); + r + } else { + p + } + } + + fn union(&mut self, a: String, b: String) { + let ra = self.find(a); + let rb = self.find(b); + if ra != rb { + self.parent.insert(ra, rb); + } + } +} + +fn col_key(e: &Expr) -> Option { + match e { + Expr::Column(c) => Some(format!("{}", c)), + Expr::Cast(c) => col_key(&c.expr), + Expr::Alias(a) => col_key(&a.expr), + _ => None, + } +} + +fn dedupe_join_on(on: Vec<(Expr, Expr)>) -> Vec<(Expr, Expr)> { + let mut uf = UnionFind::default(); + let mut seen: HashSet<(String, String)> = HashSet::new(); + let mut result = Vec::with_capacity(on.len()); + for (l, r) in on.into_iter() { + if let (Some(kl), Some(kr)) = (col_key(&l), col_key(&r)) { + let a = uf.find(kl); + let b = uf.find(kr); + if a == b { + continue; + } + let key = if a <= b { + (a.clone(), b.clone()) + } else { + (b.clone(), a.clone()) + }; + if seen.insert(key) { + uf.union(a, b); + result.push((l, r)); + } + } else { + result.push((l, r)); + } + } + result +} + +fn residual_minus_on(filter: Option, on: &[(Expr, Expr)]) -> Option { + let filter = filter?; + let exprs = split_conjunction_owned(filter); + + let mut on_set: HashSet<(String, String)> = HashSet::new(); + for (l, r) in on { + let (a, b) = canonical_pair(l, r); + on_set.insert((a.clone(), b.clone())); + } + + let remaining: Vec = exprs + .into_iter() + .filter(|e| !is_self_equality(e)) + .filter(|e| { + let inner = match e { + Expr::Alias(alias) => alias.expr.as_ref(), + _ => e, + }; + if let Expr::BinaryExpr(BinaryExpr { + left, + op: Operator::Eq, + right, + }) = inner + { + let (a, b) = canonical_pair(left, right); + !on_set.contains(&(a, b)) + } else { + true + } + }) + .collect(); + + remaining.into_iter().reduce(Expr::and) +} + +fn canonical_pair(left: &Expr, right: &Expr) -> (String, String) { + let l = canonical_str(left); + let r = canonical_str(right); + if l <= r { + (l, r) + } else { + (r, l) + } +} + +fn canonical_str(expr: &Expr) -> String { + match expr { + Expr::Alias(alias) => canonical_str(&alias.expr), + _ => format!("{}", expr), + } +} + +fn is_self_equality(expr: &Expr) -> bool { + let inner = match expr { + Expr::Alias(alias) => alias.expr.as_ref(), + _ => expr, + }; + match inner { + Expr::BinaryExpr(BinaryExpr { + left, + op: Operator::Eq, + right, + }) => left.as_ref() == right.as_ref(), + _ => false, + } +} + #[cfg(test)] mod tests { use super::*; use crate::assert_optimized_plan_eq_display_indent_snapshot; use crate::test::*; + use crate::{Optimizer, OptimizerContext}; use arrow::datatypes::DataType; + use datafusion_common::NullEquality; use datafusion_expr::{ - col, lit, logical_plan::builder::LogicalPlanBuilder, JoinType, + col, lit, logical_plan::builder::LogicalPlanBuilder, + logical_plan::JoinConstraint, JoinType, }; use std::sync::Arc; + fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + macro_rules! assert_optimized_plan_equal { ( $plan:expr, @@ -219,6 +362,36 @@ mod tests { ) } + #[test] + fn residual_minus_on_removes_symmetric_dup() -> Result<()> { + let left = test_table_scan_with_name("l")?; + let right = test_table_scan_with_name("r")?; + let on = vec![(col("l.a"), col("r.a"))]; + let filter = Some(col("r.a").eq(col("l.a")).and(col("l.a").eq(col("l.a")))); + let join = Join::try_new( + Arc::new(left), + Arc::new(right), + on, + filter, + JoinType::Inner, + JoinConstraint::On, + NullEquality::NullEqualsNull, + )?; + let optimizer = + Optimizer::with_rules(vec![Arc::new(ExtractEquijoinPredicate::new())]); + let optimized = optimizer.optimize( + LogicalPlan::Join(join), + &OptimizerContext::new(), + observe, + )?; + if let LogicalPlan::Join(j) = optimized { + assert!(j.filter.is_none()); + } else { + panic!("expected join"); + } + Ok(()) + } + #[test] fn join_with_only_none_equi_predicate() -> Result<()> { let t1 = test_table_scan_with_name("t1")?; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index e07dc767fb76..9ddc00266e76 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -35,6 +35,7 @@ use datafusion_expr::expr_rewriter::replace_col; use datafusion_expr::logical_plan::{Join, JoinType, LogicalPlan, TableScan, Union}; use datafusion_expr::utils::{ conjunction, expr_to_columns, split_conjunction, split_conjunction_owned, + INFERRED_PREDICATE_ALIAS, }; use datafusion_expr::{ and, or, BinaryExpr, Expr, Filter, Operator, Projection, TableProviderFilterPushDown, @@ -463,7 +464,11 @@ fn push_down_all_join( } else if right_preserved && checker.is_right_only(&predicate) { right_push.push(predicate); } else { - join_conditions.push(predicate); + // Mark inferred predicates so subsequent optimization passes do not + // treat them as additional equijoin keys. They should remain as + // residual join filters to enable dynamic filtering without + // widening the join key set. + join_conditions.push(predicate.alias(INFERRED_PREDICATE_ALIAS)); } } @@ -1487,6 +1492,53 @@ mod tests { }}; } + #[test] + fn inferred_predicate_stays_in_filter() -> Result<()> { + use datafusion_common::NullEquality; + use datafusion_expr::logical_plan::JoinConstraint; + + let left = test_table_scan_with_name("l")?; + let right = test_table_scan_with_name("r")?; + + let join = Join::try_new( + Arc::new(left), + Arc::new(right), + vec![], + None, + JoinType::Left, + JoinConstraint::On, + NullEquality::NullEqualsNull, + )?; + + let inferred = col("l.a").eq(col("r.a")); + let Transformed { data: plan, .. } = + push_down_all_join(vec![], vec![inferred], join, vec![])?; + + let join = match plan { + LogicalPlan::Join(j) => j, + _ => panic!("expected join"), + }; + + assert!(join.on.is_empty()); + let filter = join.filter.clone().expect("expected filter"); + match filter { + Expr::Alias(alias) => assert_eq!(alias.name, INFERRED_PREDICATE_ALIAS), + _ => panic!("expected aliased filter"), + } + + let optimizer = Optimizer::with_rules(vec![ + Arc::new(SimplifyExpressions::new()), + Arc::new(crate::extract_equijoin_predicate::ExtractEquijoinPredicate::new()), + ]); + let optimized = optimizer.optimize(LogicalPlan::Join(join), &OptimizerContext::new(), observe)?; + if let LogicalPlan::Join(j) = optimized { + assert!(j.on.is_empty()); + } else { + panic!("expected join"); + } + Ok(()) + } + #[test] fn filter_before_projection() -> Result<()> { let table_scan = test_table_scan()?; diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 88dcd4c523cf..39a3e5c1a3b4 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -21,6 +21,7 @@ //! according to the configuration), this rule increases partition counts in //! the physical plan. +use std::collections::HashSet; use std::fmt::Debug; use std::sync::Arc; @@ -896,7 +897,16 @@ fn add_hash_on_top( return Ok(input); } - let dist = Distribution::HashPartitioned(hash_exprs); + let mut seen = HashSet::new(); + let mut exprs = Vec::new(); + for e in hash_exprs.into_iter() { + let key = format!("{}", e); + if seen.insert(key) { + exprs.push(e); + } + } + + let dist = Distribution::HashPartitioned(exprs.clone()); let satisfied = input .plan .output_partitioning() From 652708533955fdb531d4fa2277aa24513458fce0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 12 Aug 2025 19:05:13 +0800 Subject: [PATCH 10/72] Refactor: Remove inferred predicate alias handling and related tests --- datafusion/core/src/physical_planner.rs | 73 +------ datafusion/expr/src/expr.rs | 9 +- datafusion/expr/src/utils.rs | 60 +----- .../src/extract_equijoin_predicate.rs | 185 +----------------- datafusion/optimizer/src/push_down_filter.rs | 148 +------------- .../src/enforce_distribution.rs | 12 +- .../join_disable_repartition_joins.slt | 23 +-- datafusion/sqllogictest/test_files/joins.slt | 10 +- 8 files changed, 32 insertions(+), 488 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 427b423be9b8..1021abc9e47b 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -18,7 +18,7 @@ //! Planner for [`LogicalPlan`] to [`ExecutionPlan`] use std::borrow::Cow; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use std::sync::Arc; use crate::datasource::file_format::file_type_to_format; @@ -151,46 +151,6 @@ pub trait ExtensionPlanner { ) -> Result>>; } -fn expr_canonical_string(expr: &Arc) -> String { - format!("{}", expr) -} - -fn dedupe_join_key_pairs( - left: Vec>, - right: Vec>, -) -> (Vec>, Vec>) { - assert_eq!(left.len(), right.len()); - let mut seen = HashSet::new(); - let mut out_left = Vec::new(); - let mut out_right = Vec::new(); - for (l, r) in left.into_iter().zip(right.into_iter()) { - let l_s = expr_canonical_string(&l); - let r_s = expr_canonical_string(&r); - let key = if l_s <= r_s { - format!("{}={}", l_s, r_s) - } else { - format!("{}={}", r_s, l_s) - }; - if seen.insert(key) { - out_left.push(l); - out_right.push(r); - } - } - (out_left, out_right) -} - -fn dedupe_partition_exprs(exprs: Vec>) -> Vec> { - let mut seen = HashSet::new(); - let mut out = Vec::new(); - for e in exprs.into_iter() { - let key = expr_canonical_string(&e); - if seen.insert(key) { - out.push(e); - } - } - out -} - /// Default single node physical query planner that converts a /// `LogicalPlan` to an `ExecutionPlan` suitable for execution. /// @@ -898,7 +858,6 @@ impl DefaultPhysicalPlanner { ) }) .collect::>>()?; - let runtime_expr = dedupe_partition_exprs(runtime_expr); Partitioning::Hash(runtime_expr, *n) } LogicalPartitioning::DistributeBy(_) => { @@ -1129,10 +1088,6 @@ impl DefaultPhysicalPlanner { Ok((l, r)) }) .collect::>()?; - let (left_keys, right_keys): (Vec<_>, Vec<_>) = join_on.into_iter().unzip(); - let (left_keys, right_keys) = dedupe_join_key_pairs(left_keys, right_keys); - let join_on: join_utils::JoinOn = - left_keys.into_iter().zip(right_keys).collect(); let join_filter = match filter { Some(expr) => { @@ -2390,7 +2345,6 @@ impl<'a> OptimizationInvariantChecker<'a> { Ok(()) } - } impl<'n> TreeNodeVisitor<'n> for OptimizationInvariantChecker<'_> { @@ -3566,29 +3520,4 @@ digraph { Ok(()) } - - #[test] - fn test_dedupe_join_key_pairs() { - let l1: Arc = Arc::new(Column::new("a", 0)); - let r1: Arc = Arc::new(Column::new("b", 0)); - let l2: Arc = Arc::new(Column::new("b", 0)); - let r2: Arc = Arc::new(Column::new("a", 0)); - let (left, right) = dedupe_join_key_pairs(vec![l1.clone(), l2], vec![r1.clone(), r2]); - assert_eq!(left.len(), 1); - assert_eq!(format!("{}", left[0]), "a@0"); - assert_eq!(format!("{}", right[0]), "b@0"); - } - - #[test] - fn test_dedupe_partition_exprs() { - let exprs: Vec> = vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]; - let result = dedupe_partition_exprs(exprs); - assert_eq!(result.len(), 2); - assert_eq!(format!("{}", result[0]), "a@0"); - assert_eq!(format!("{}", result[1]), "b@1"); - } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 17019a125681..bb376b591696 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -38,7 +38,6 @@ use datafusion_common::tree_node::{ use datafusion_common::{ Column, DFSchema, HashMap, Result, ScalarValue, Spans, TableReference, }; -use crate::utils::INFERRED_PREDICATE_ALIAS; use datafusion_functions_window_common::field::WindowUDFFieldArgs; use sqlparser::ast::{ display_comma_separated, ExceptSelectItem, ExcludeSelectItem, IlikeSelectItem, @@ -3184,13 +3183,7 @@ pub const UNNEST_COLUMN_PREFIX: &str = "UNNEST"; impl Display for Expr { fn fmt(&self, f: &mut Formatter) -> fmt::Result { match self { - Expr::Alias(Alias { expr, name, .. }) => { - if name == INFERRED_PREDICATE_ALIAS { - write!(f, "{expr}") - } else { - write!(f, "{expr} AS {name}") - } - } + Expr::Alias(Alias { expr, name, .. }) => write!(f, "{expr} AS {name}"), Expr::Column(c) => write!(f, "{c}"), Expr::OuterReferenceColumn(_, c) => { write!(f, "{OUTER_REFERENCE_COLUMN_PREFIX}({c})") diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index cbbe42db6f51..80ad0f87846a 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -28,15 +28,6 @@ use crate::{ }; use datafusion_expr_common::signature::{Signature, TypeSignature}; -/// Alias used to mark inferred join predicates that should remain join filters -/// and not be converted into additional equijoin keys during optimization. -pub const INFERRED_PREDICATE_ALIAS: &str = "__datafusion_inferred_join_predicate"; - -/// Returns true if expr is an alias used to mark inferred join predicates. -pub fn is_inferred_alias(expr: &Expr) -> bool { - matches!(expr, Expr::Alias(Alias { name, .. }) if name == INFERRED_PREDICATE_ALIAS) -} - use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, @@ -955,9 +946,7 @@ fn split_conjunction_impl<'a>(expr: &'a Expr, mut exprs: Vec<&'a Expr>) -> Vec<& let exprs = split_conjunction_impl(left, exprs); split_conjunction_impl(right, exprs) } - Expr::Alias(alias) if !is_inferred_alias(expr) => { - split_conjunction_impl(&alias.expr, exprs) - } + Expr::Alias(Alias { expr, .. }) => split_conjunction_impl(expr, exprs), other => { exprs.push(other); exprs @@ -981,11 +970,7 @@ pub fn iter_conjunction(expr: &Expr) -> impl Iterator { stack.push(right); stack.push(left); } - Expr::Alias(Alias { expr, name, .. }) - if name != INFERRED_PREDICATE_ALIAS => - { - stack.push(expr); - } + Expr::Alias(Alias { expr, .. }) => stack.push(expr), other => return Some(other), } } @@ -1009,11 +994,7 @@ pub fn iter_conjunction_owned(expr: Expr) -> impl Iterator { stack.push(*right); stack.push(*left); } - Expr::Alias(Alias { expr, name, .. }) - if name != INFERRED_PREDICATE_ALIAS => - { - stack.push(*expr); - } + Expr::Alias(Alias { expr, .. }) => stack.push(*expr), other => return Some(other), } } @@ -1082,13 +1063,8 @@ fn split_binary_owned_impl( let exprs = split_binary_owned_impl(*left, operator, exprs); split_binary_owned_impl(*right, operator, exprs) } - Expr::Alias(alias) => { - if is_inferred_alias(&Expr::Alias(alias.clone())) { - exprs.push(Expr::Alias(alias)); - exprs - } else { - split_binary_owned_impl(*alias.expr, operator, exprs) - } + Expr::Alias(Alias { expr, .. }) => { + split_binary_owned_impl(*expr, operator, exprs) } other => { exprs.push(other); @@ -1114,9 +1090,7 @@ fn split_binary_impl<'a>( let exprs = split_binary_impl(left, operator, exprs); split_binary_impl(right, operator, exprs) } - Expr::Alias(alias) if !is_inferred_alias(expr) => { - split_binary_impl(&alias.expr, operator, exprs) - } + Expr::Alias(Alias { expr, .. }) => split_binary_impl(expr, operator, exprs), other => { exprs.push(other); exprs @@ -1674,17 +1648,6 @@ mod tests { assert_eq!(result, vec![&expr1, &expr2]); } - #[test] - fn test_split_conjunction_inferred_alias() { - let expr = col("a").eq(col("b")).alias(INFERRED_PREDICATE_ALIAS); - let result = split_conjunction(&expr); - assert_eq!(result.len(), 1); - match result[0] { - Expr::Alias(alias) => assert_eq!(alias.name, INFERRED_PREDICATE_ALIAS), - _ => panic!("expected alias"), - } - } - #[test] fn test_split_conjunction_or() { let expr = col("a").eq(lit(5)).or(col("b")); @@ -1742,17 +1705,6 @@ mod tests { ); } - #[test] - fn test_split_conjunction_owned_inferred_alias() { - let expr = col("a").eq(col("b")).alias(INFERRED_PREDICATE_ALIAS); - let result = split_conjunction_owned(expr); - assert_eq!(result.len(), 1); - match &result[0] { - Expr::Alias(alias) => assert_eq!(alias.name, INFERRED_PREDICATE_ALIAS), - _ => panic!("expected alias"), - } - } - #[test] fn test_conjunction_empty() { assert_eq!(conjunction(vec![]), None); diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index 63515f1b0bc5..55cf33ef4304 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -22,9 +22,8 @@ use datafusion_common::tree_node::Transformed; use datafusion_common::DFSchema; use datafusion_common::Result; use datafusion_expr::utils::split_conjunction_owned; -use datafusion_expr::utils::{can_hash, find_valid_equijoin_key_pair, is_inferred_alias}; +use datafusion_expr::utils::{can_hash, find_valid_equijoin_key_pair}; use datafusion_expr::{BinaryExpr, Expr, ExprSchemable, Join, LogicalPlan, Operator}; -use std::collections::{HashMap, HashSet}; // equijoin predicate type EquijoinPredicate = (Expr, Expr); @@ -82,18 +81,14 @@ impl OptimizerRule for ExtractEquijoinPredicate { let right_schema = right.schema(); let (equijoin_predicates, non_equijoin_expr) = split_eq_and_noneq_join_predicate(expr, left_schema, right_schema)?; - let has_new_keys = !equijoin_predicates.is_empty(); - on.extend(equijoin_predicates); - on = dedupe_join_on(on); - let filter = residual_minus_on(non_equijoin_expr, &on); - - if has_new_keys { + if !equijoin_predicates.is_empty() { + on.extend(equijoin_predicates); Ok(Transformed::yes(LogicalPlan::Join(Join { left, right, on, - filter, + filter: non_equijoin_expr, join_type, join_constraint, schema, @@ -104,7 +99,7 @@ impl OptimizerRule for ExtractEquijoinPredicate { left, right, on, - filter, + filter: non_equijoin_expr, join_type, join_constraint, schema, @@ -127,10 +122,6 @@ fn split_eq_and_noneq_join_predicate( let mut accum_join_keys: Vec<(Expr, Expr)> = vec![]; let mut accum_filters: Vec = vec![]; for expr in exprs { - if is_inferred_alias(&expr) { - accum_filters.push(expr); - continue; - } match expr { Expr::BinaryExpr(BinaryExpr { ref left, @@ -161,151 +152,17 @@ fn split_eq_and_noneq_join_predicate( Ok((accum_join_keys, result_filter)) } -#[derive(Default)] -struct UnionFind { - parent: HashMap, -} - -impl UnionFind { - fn find(&mut self, x: String) -> String { - let p = self.parent.get(&x).cloned().unwrap_or_else(|| x.clone()); - if p != x { - let r = self.find(p.clone()); - self.parent.insert(x, r.clone()); - r - } else { - p - } - } - - fn union(&mut self, a: String, b: String) { - let ra = self.find(a); - let rb = self.find(b); - if ra != rb { - self.parent.insert(ra, rb); - } - } -} - -fn col_key(e: &Expr) -> Option { - match e { - Expr::Column(c) => Some(format!("{}", c)), - Expr::Cast(c) => col_key(&c.expr), - Expr::Alias(a) => col_key(&a.expr), - _ => None, - } -} - -fn dedupe_join_on(on: Vec<(Expr, Expr)>) -> Vec<(Expr, Expr)> { - let mut uf = UnionFind::default(); - let mut seen: HashSet<(String, String)> = HashSet::new(); - let mut result = Vec::with_capacity(on.len()); - for (l, r) in on.into_iter() { - if let (Some(kl), Some(kr)) = (col_key(&l), col_key(&r)) { - let a = uf.find(kl); - let b = uf.find(kr); - if a == b { - continue; - } - let key = if a <= b { - (a.clone(), b.clone()) - } else { - (b.clone(), a.clone()) - }; - if seen.insert(key) { - uf.union(a, b); - result.push((l, r)); - } - } else { - result.push((l, r)); - } - } - result -} - -fn residual_minus_on(filter: Option, on: &[(Expr, Expr)]) -> Option { - let filter = filter?; - let exprs = split_conjunction_owned(filter); - - let mut on_set: HashSet<(String, String)> = HashSet::new(); - for (l, r) in on { - let (a, b) = canonical_pair(l, r); - on_set.insert((a.clone(), b.clone())); - } - - let remaining: Vec = exprs - .into_iter() - .filter(|e| !is_self_equality(e)) - .filter(|e| { - let inner = match e { - Expr::Alias(alias) => alias.expr.as_ref(), - _ => e, - }; - if let Expr::BinaryExpr(BinaryExpr { - left, - op: Operator::Eq, - right, - }) = inner - { - let (a, b) = canonical_pair(left, right); - !on_set.contains(&(a, b)) - } else { - true - } - }) - .collect(); - - remaining.into_iter().reduce(Expr::and) -} - -fn canonical_pair(left: &Expr, right: &Expr) -> (String, String) { - let l = canonical_str(left); - let r = canonical_str(right); - if l <= r { - (l, r) - } else { - (r, l) - } -} - -fn canonical_str(expr: &Expr) -> String { - match expr { - Expr::Alias(alias) => canonical_str(&alias.expr), - _ => format!("{}", expr), - } -} - -fn is_self_equality(expr: &Expr) -> bool { - let inner = match expr { - Expr::Alias(alias) => alias.expr.as_ref(), - _ => expr, - }; - match inner { - Expr::BinaryExpr(BinaryExpr { - left, - op: Operator::Eq, - right, - }) => left.as_ref() == right.as_ref(), - _ => false, - } -} - #[cfg(test)] mod tests { use super::*; use crate::assert_optimized_plan_eq_display_indent_snapshot; use crate::test::*; - use crate::{Optimizer, OptimizerContext}; use arrow::datatypes::DataType; - use datafusion_common::NullEquality; use datafusion_expr::{ - col, lit, logical_plan::builder::LogicalPlanBuilder, - logical_plan::JoinConstraint, JoinType, + col, lit, logical_plan::builder::LogicalPlanBuilder, JoinType, }; use std::sync::Arc; - fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} - macro_rules! assert_optimized_plan_equal { ( $plan:expr, @@ -362,36 +219,6 @@ mod tests { ) } - #[test] - fn residual_minus_on_removes_symmetric_dup() -> Result<()> { - let left = test_table_scan_with_name("l")?; - let right = test_table_scan_with_name("r")?; - let on = vec![(col("l.a"), col("r.a"))]; - let filter = Some(col("r.a").eq(col("l.a")).and(col("l.a").eq(col("l.a")))); - let join = Join::try_new( - Arc::new(left), - Arc::new(right), - on, - filter, - JoinType::Inner, - JoinConstraint::On, - NullEquality::NullEqualsNull, - )?; - let optimizer = - Optimizer::with_rules(vec![Arc::new(ExtractEquijoinPredicate::new())]); - let optimized = optimizer.optimize( - LogicalPlan::Join(join), - &OptimizerContext::new(), - observe, - )?; - if let LogicalPlan::Join(j) = optimized { - assert!(j.filter.is_none()); - } else { - panic!("expected join"); - } - Ok(()) - } - #[test] fn join_with_only_none_equi_predicate() -> Result<()> { let t1 = test_table_scan_with_name("t1")?; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 9ddc00266e76..35ec7d074d5f 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -35,7 +35,6 @@ use datafusion_expr::expr_rewriter::replace_col; use datafusion_expr::logical_plan::{Join, JoinType, LogicalPlan, TableScan, Union}; use datafusion_expr::utils::{ conjunction, expr_to_columns, split_conjunction, split_conjunction_owned, - INFERRED_PREDICATE_ALIAS, }; use datafusion_expr::{ and, or, BinaryExpr, Expr, Filter, Operator, Projection, TableProviderFilterPushDown, @@ -454,21 +453,12 @@ fn push_down_all_join( } } - // For inferred predicates, if they can not be pushed to either side of the join - // (for example because the corresponding input is not preserved by the join - // type), retain them as join filters so they can still participate in - // dynamic filter pushdown. + // For infer predicates, if they can not push through join, just drop them for predicate in inferred_join_predicates { if left_preserved && checker.is_left_only(&predicate) { left_push.push(predicate); } else if right_preserved && checker.is_right_only(&predicate) { right_push.push(predicate); - } else { - // Mark inferred predicates so subsequent optimization passes do not - // treat them as additional equijoin keys. They should remain as - // residual join filters to enable dynamic filtering without - // widening the join key set. - join_conditions.push(predicate.alias(INFERRED_PREDICATE_ALIAS)); } } @@ -1492,53 +1482,6 @@ mod tests { }}; } - #[test] - fn inferred_predicate_stays_in_filter() -> Result<()> { - use datafusion_common::NullEquality; - use datafusion_expr::logical_plan::JoinConstraint; - - let left = test_table_scan_with_name("l")?; - let right = test_table_scan_with_name("r")?; - - let join = Join::try_new( - Arc::new(left), - Arc::new(right), - vec![], - None, - JoinType::Left, - JoinConstraint::On, - NullEquality::NullEqualsNull, - )?; - - let inferred = col("l.a").eq(col("r.a")); - let Transformed { data: plan, .. } = - push_down_all_join(vec![], vec![inferred], join, vec![])?; - - let join = match plan { - LogicalPlan::Join(j) => j, - _ => panic!("expected join"), - }; - - assert!(join.on.is_empty()); - let filter = join.filter.clone().expect("expected filter"); - match filter { - Expr::Alias(alias) => assert_eq!(alias.name, INFERRED_PREDICATE_ALIAS), - _ => panic!("expected aliased filter"), - } - - let optimizer = Optimizer::with_rules(vec![ - Arc::new(SimplifyExpressions::new()), - Arc::new(crate::extract_equijoin_predicate::ExtractEquijoinPredicate::new()), - ]); - let optimized = optimizer.optimize(LogicalPlan::Join(join), &OptimizerContext::new(), observe)?; - if let LogicalPlan::Join(j) = optimized { - assert!(j.on.is_empty()); - } else { - panic!("expected join"); - } - Ok(()) - } - #[test] fn filter_before_projection() -> Result<()> { let table_scan = test_table_scan()?; @@ -2746,7 +2689,7 @@ mod tests { assert_optimized_plan_equal!( plan, @r" - Left Join: Using test.a = test2.a Filter: test2.a <= Int64(1) + Left Join: Using test.a = test2.a TableScan: test, full_filters=[test.a <= Int64(1)] Projection: test2.a TableScan: test2 @@ -2787,7 +2730,7 @@ mod tests { assert_optimized_plan_equal!( plan, @r" - Right Join: Using test.a = test2.a Filter: test.a <= Int64(1) + Right Join: Using test.a = test2.a TableScan: test Projection: test2.a TableScan: test2, full_filters=[test2.a <= Int64(1)] @@ -2966,7 +2909,7 @@ mod tests { assert_optimized_plan_equal!( plan, @r" - Left Join: test.a = test2.a Filter: test2.a > UInt32(1) AND test.a > UInt32(1) AND test.b < test2.b + Left Join: test.a = test2.a Filter: test.a > UInt32(1) AND test.b < test2.b Projection: test.a, test.b, test.c TableScan: test Projection: test2.a, test2.b, test2.c @@ -3021,89 +2964,6 @@ mod tests { ) } - /// A filter on the preserved(left) side of a left join should be - /// converted into a join filter on the right side via dynamic filter pushdown - #[test] - fn left_join_dynamic_filter_pushdown() -> Result<()> { - let left = test_table_scan()?; - let right = test_table_scan_with_name("test2")?; - - let filter = col("test.a").gt(lit(1u32)); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Left, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - assert_optimized_plan_equal!( - plan, - @r" - Left Join: test.a = test2.a Filter: test2.a > UInt32(1) - TableScan: test, full_filters=[test.a > UInt32(1)] - TableScan: test2 - " - ) - } - - /// A filter on the preserved(right) side of a right join should be - /// converted into a join filter on the left side via dynamic filter pushdown - #[test] - fn right_join_dynamic_filter_pushdown() -> Result<()> { - let left = test_table_scan()?; - let right = test_table_scan_with_name("test2")?; - - let filter = col("test2.a").gt(lit(1u32)); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Right, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - assert_optimized_plan_equal!( - plan, - @r" - Right Join: test.a = test2.a Filter: test.a > UInt32(1) - TableScan: test - TableScan: test2, full_filters=[test2.a > UInt32(1)] - " - ) - } - - /// Filters that do not restrict nulls should not generate dynamic filters - #[test] - fn left_join_dynamic_filter_pushdown_with_nulls() -> Result<()> { - let left = test_table_scan()?; - let right = test_table_scan_with_name("test2")?; - - let filter = col("test.a").is_null(); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Left, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - assert_optimized_plan_equal!( - plan, - @r" - Left Join: test.a = test2.a - TableScan: test, full_filters=[test.a IS NULL] - TableScan: test2 - " - ) - } - /// single table predicate parts of ON condition should not be pushed #[test] fn full_join_on_with_filter() -> Result<()> { diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 39a3e5c1a3b4..88dcd4c523cf 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -21,7 +21,6 @@ //! according to the configuration), this rule increases partition counts in //! the physical plan. -use std::collections::HashSet; use std::fmt::Debug; use std::sync::Arc; @@ -897,16 +896,7 @@ fn add_hash_on_top( return Ok(input); } - let mut seen = HashSet::new(); - let mut exprs = Vec::new(); - for e in hash_exprs.into_iter() { - let key = format!("{}", e); - if seen.insert(key) { - exprs.push(e); - } - } - - let dist = Distribution::HashPartitioned(exprs.clone()); + let dist = Distribution::HashPartitioned(hash_exprs); let satisfied = input .plan .output_partitioning() diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index c214778eea03..a1efc1317b4a 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -89,25 +89,20 @@ logical_plan 02)--Projection: t2.a AS a2, t2.b 03)----RightSemi Join: t1.d = t2.d, t1.c = t2.c 04)------SubqueryAlias: t1 -05)--------Filter: annotated_data.d = Int32(3) -06)----------TableScan: annotated_data projection=[c, d], partial_filters=[annotated_data.d = Int32(3)] -07)------SubqueryAlias: t2 -08)--------Filter: annotated_data.d = Int32(3) -09)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] +05)--------TableScan: annotated_data projection=[c, d] +06)------SubqueryAlias: t2 +07)--------Filter: annotated_data.d = Int32(3) +08)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] physical_plan 01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -05)--------CoalescePartitionsExec -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: d@1 = 3 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true -10)--------CoalesceBatchesExec: target_batch_size=8192 -11)----------FilterExec: d@3 = 3 -12)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true +06)--------CoalesceBatchesExec: target_batch_size=8192 +07)----------FilterExec: d@3 = 3 +08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 4efa07d61c7d..e7beec17e977 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -5148,12 +5148,10 @@ WHERE k1 < 0 physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] -03)----CoalesceBatchesExec: target_batch_size=3 -04)------FilterExec: k2@0 < 0 -05)--------DataSourceExec: partitions=1, partition_sizes=[0] -06)----CoalesceBatchesExec: target_batch_size=3 -07)------FilterExec: k1@0 < 0 -08)--------DataSourceExec: partitions=1, partition_sizes=[10000] +03)----DataSourceExec: partitions=1, partition_sizes=[0] +04)----CoalesceBatchesExec: target_batch_size=3 +05)------FilterExec: k1@0 < 0 +06)--------DataSourceExec: partitions=1, partition_sizes=[10000] query II SELECT * From b258eadf810f384c244e650a7cdd614f051469f7 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Tue, 12 Aug 2025 19:15:18 +0800 Subject: [PATCH 11/72] fix fmt errors --- .../physical_optimizer/filter_pushdown/mod.rs | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 1ca940e597f3..630ca4557753 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -32,7 +32,9 @@ use datafusion::{ prelude::{ParquetReadOptions, SessionConfig, SessionContext}, scalar::ScalarValue, }; -use datafusion_common::{assert_contains, assert_not_contains, config::ConfigOptions, JoinType}; +use datafusion_common::{ + assert_contains, assert_not_contains, config::ConfigOptions, JoinType, +}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::ScalarUDF; use datafusion_functions::math::random::RandomFunc; @@ -1134,8 +1136,14 @@ fn build_join_with_dynamic_filter( .build(); let on = vec![ - (col("a", &left_schema).unwrap(), col("a", &right_schema).unwrap()), - (col("b", &left_schema).unwrap(), col("b", &right_schema).unwrap()), + ( + col("a", &left_schema).unwrap(), + col("a", &right_schema).unwrap(), + ), + ( + col("b", &left_schema).unwrap(), + col("b", &right_schema).unwrap(), + ), ]; Arc::new( From 4a497ff95023263a9bc3b61a226cbbd158d5e548 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 16:54:34 +0800 Subject: [PATCH 12/72] docs: improve documentation for dynamic filter pushdown configuration --- datafusion/common/src/config.rs | 27 +++++++++++++++------------ 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 981f331d7630..1636c1654797 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -725,18 +725,21 @@ config_namespace! { /// during aggregations, if possible pub enable_topk_aggregation: bool, default = true - /// When set to true attempts to push down dynamic filters generated by operators - /// into the file scan phase. For example, for a query such as - /// `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt - /// to push down the current top 10 timestamps that the TopK operator references - /// into the file scans. This means that if we already have 10 timestamps in the - /// year 2025 any files that only have timestamps in the year 2024 can be skipped / - /// pruned at various stages in the scan. Dynamic filters are also produced by - /// left, right, semi, and anti joins, allowing DataFusion to prune the probe side - /// during execution. Full joins are not supported. For example, - /// `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` - /// will only read `fact` rows whose `id` values match `dim` rows where - /// `region = 'US'`. + /// When set to true, attempts to push down dynamic filters generated by operators + /// into the file scan phase. For example, for a query such as + /// `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt + /// to push down the current top 10 timestamps that the TopK operator references + /// into the file scans, allowing files without matching values to be skipped/pruned. + /// + /// Dynamic filters are also produced by join operators with equi-join predicates. + /// For hash joins, dynamic filters are applied to the probe side (the stream that is scanned), + /// which is the right input for Inner and Left joins and the left input for Right joins; + /// semi and anti joins prune the left/probe input. Full joins are not supported. + /// + /// For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` + /// will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. + /// + /// Note that non-equi join predicates do not generate dynamic filters. pub enable_dynamic_filter_pushdown: bool, default = true /// When set to true, the optimizer will insert filters before a join between From f78b8a59a3317b8b8ead2a3e95ce4f6d51c2331f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 16:56:26 +0800 Subject: [PATCH 13/72] test: add dynamic filter pushdown tests for right semi and right anti joins --- .../physical_optimizer/filter_pushdown/mod.rs | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 630ca4557753..d1b210ce4700 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1241,6 +1241,46 @@ async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { ); } +#[tokio::test] +async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::RightSemi, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_not_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + ); +} + +#[tokio::test] +async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::RightAnti, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_not_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + ); +} + #[tokio::test] async fn test_hashjoin_full_dynamic_filter_no_pushdown() { let plan = build_join_with_dynamic_filter(JoinType::Full, true, true); From e336ad935840ac00a6ad577f4c8b7170296d1826 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 17:44:26 +0800 Subject: [PATCH 14/72] Enhance documentation for dynamic filters in joins Updated comments in `config.rs` to clarify the application of dynamic filters in hash joins for probe sides. Added details regarding Left, Right, Semi, and Anti joins, as well as explicit notes on the non-support for full joins and non-equi join predicates. --- datafusion/common/src/config.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 981f331d7630..e0435f595b41 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -733,7 +733,11 @@ config_namespace! { /// year 2025 any files that only have timestamps in the year 2024 can be skipped / /// pruned at various stages in the scan. Dynamic filters are also produced by /// left, right, semi, and anti joins, allowing DataFusion to prune the probe side - /// during execution. Full joins are not supported. For example, + /// during execution. For hash joins, dynamic filters are applied to the **probe** + /// side (the stream that is scanned), which is the right input for `Inner` and + /// `Left` joins and the left input for `Right` joins; semi and anti joins prune the + /// left/probe input. Full joins are not supported and non-equi join predicates do + /// **not** generate dynamic filters. For example, /// `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` /// will only read `fact` rows whose `id` values match `dim` rows where /// `region = 'US'`. From d24cd4f3440534dddb42d034eb114f6e1d73acd2 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 17:44:30 +0800 Subject: [PATCH 15/72] Add tests for disabled dynamic filter pushdown on hash joins - Implemented asynchronous tests to verify the behavior of dynamic filter pushdown for left, right, and anti joins when the filter pushdown is explicitly disabled in the configuration. - Added assertions to ensure that `DynamicFilterPhysicalExpr` is not present in the optimized plans when dynamic filter pushdown is disabled. - Included tests for both semi and anti joins to ensure comprehensive coverage of dynamic filtering functionality. --- .../physical_optimizer/filter_pushdown/mod.rs | 79 +++++++++++++++++++ 1 file changed, 79 insertions(+) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 630ca4557753..9b11f7296040 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1181,6 +1181,19 @@ async fn test_hashjoin_left_dynamic_filter_pushdown() { ); } +#[tokio::test] +async fn test_hashjoin_left_dynamic_filter_pushdown_disabled() { + let plan = build_join_with_dynamic_filter(JoinType::Left, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = false; + config.optimizer.enable_dynamic_filter_pushdown = false; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); +} + #[tokio::test] async fn test_hashjoin_right_dynamic_filter_pushdown() { let plan = build_join_with_dynamic_filter(JoinType::Right, true, true); @@ -1201,6 +1214,19 @@ async fn test_hashjoin_right_dynamic_filter_pushdown() { ); } +#[tokio::test] +async fn test_hashjoin_right_dynamic_filter_pushdown_disabled() { + let plan = build_join_with_dynamic_filter(JoinType::Right, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = false; + config.optimizer.enable_dynamic_filter_pushdown = false; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); +} + #[tokio::test] async fn test_hashjoin_left_semi_dynamic_filter_pushdown() { let plan = build_join_with_dynamic_filter(JoinType::LeftSemi, true, true); @@ -1241,6 +1267,46 @@ async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { ); } +#[tokio::test] +async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::RightSemi, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_not_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + ); +} + +#[tokio::test] +async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter(JoinType::RightAnti, true, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_not_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + ); +} + #[tokio::test] async fn test_hashjoin_full_dynamic_filter_no_pushdown() { let plan = build_join_with_dynamic_filter(JoinType::Full, true, true); @@ -1267,6 +1333,19 @@ async fn test_hashjoin_dynamic_filter_pushdown_unsupported() { assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); } +#[tokio::test] +async fn test_hashjoin_dynamic_filter_pushdown_unsupported_left() { + let plan = build_join_with_dynamic_filter(JoinType::Right, false, true); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); +} + /// Integration test for dynamic filter pushdown with TopK. /// We use an integration test because there are complex interactions in the optimizer rules /// that the unit tests applying a single optimizer rule do not cover. From 9f4411ae065e8d6fc41aad4a97c34637a6400b36 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 17:44:36 +0800 Subject: [PATCH 16/72] Refactor dynamic filtering in HashJoinExec - Updated the `dynamic_filter` field to be an `Option>` to allow for the absence of a dynamic filter. - Adjusted the `create_dynamic_filter` method to return an `Option` and ensure it handles the `JoinSide::None` case appropriately. - Modified display logic in the `DisplayAs` trait implementation to handle dynamic filters safely. - Improved dynamic filter preservation during cloning of execution plans. - Refactored dynamic filter pushdown logic to utilize the `Option` type correctly. - Added unit tests to verify the correct behavior of the dynamic filter side determination based on join types. --- .../physical-plan/src/joins/hash_join.rs | 77 +++++++++++++------ 1 file changed, 54 insertions(+), 23 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 82a16922dd74..931cea72cbf2 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -122,7 +122,9 @@ fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { } } -// Determine which side should receive the dynamic filter +// Determine which side should receive the dynamic filter. +// Mark joins filter the opposite side of the preserved input to +// restrict rows feeding the ON clause evaluation. fn dynamic_filter_side(join_type: JoinType) -> JoinSide { let (left_preserved, right_preserved) = lr_is_preserved(join_type); let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join_type); @@ -413,7 +415,7 @@ pub struct HashJoinExec { /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, /// Dynamic filter for pushing down to the probe side - dynamic_filter: Arc, + dynamic_filter: Option>, } impl HashJoinExec { @@ -484,17 +486,17 @@ impl HashJoinExec { fn create_dynamic_filter( on: &JoinOn, join_type: JoinType, - ) -> Arc { + ) -> Option> { // Determine which side of the join the dynamic filter should target let filter_side = dynamic_filter_side(join_type); // Extract the corresponding join keys from the `on` clauses let keys: Vec<_> = match filter_side { JoinSide::Left => on.iter().map(|(l, _)| Arc::clone(l)).collect(), JoinSide::Right => on.iter().map(|(_, r)| Arc::clone(r)).collect(), - JoinSide::None => Vec::new(), + JoinSide::None => return None, }; // Initialize with a placeholder expression (true) that will be updated when the hash table is built - Arc::new(DynamicFilterPhysicalExpr::new(keys, lit(true))) + Some(Arc::new(DynamicFilterPhysicalExpr::new(keys, lit(true)))) } /// left (build) side which gets hashed @@ -742,11 +744,14 @@ impl DisplayAs for HashJoinExec { .map(|(c1, c2)| format!("({c1}, {c2})")) .collect::>() .join(", "); - let dynamic_filter_display = match self.dynamic_filter.current() { - Ok(current) if current != lit(true) => { - format!(", filter=[{current}]") - } - _ => "".to_string(), + let dynamic_filter_display = match &self.dynamic_filter { + Some(df) => match df.current() { + Ok(current) if current != lit(true) => { + format!(", filter=[{current}]") + } + _ => "".to_string(), + }, + None => "".to_string(), }; write!( f, @@ -854,7 +859,7 @@ impl ExecutionPlan for HashJoinExec { self.null_equality, )?; // Preserve the dynamic filter if it exists - new_join.dynamic_filter = Arc::clone(&self.dynamic_filter); + new_join.dynamic_filter = self.dynamic_filter.clone(); Ok(Arc::new(new_join)) } @@ -935,7 +940,8 @@ impl ExecutionPlan for HashJoinExec { need_produce_result_in_final(self.join_type), self.right().output_partitioning().partition_count(), enable_dynamic_filter_pushdown - .then_some(Arc::clone(&self.dynamic_filter)), + .then(|| self.dynamic_filter.as_ref().map(Arc::clone)) + .flatten(), on_right.clone(), )) })?, @@ -955,7 +961,8 @@ impl ExecutionPlan for HashJoinExec { need_produce_result_in_final(self.join_type), 1, enable_dynamic_filter_pushdown - .then_some(Arc::clone(&self.dynamic_filter)), + .then(|| self.dynamic_filter.as_ref().map(Arc::clone)) + .flatten(), on_right.clone(), )) } @@ -1106,20 +1113,23 @@ impl ExecutionPlan for HashJoinExec { } }; + // We only install dynamic filters after optimization to avoid planning scans + // before the hash table provides join key bounds if matches!(phase, FilterPushdownPhase::Post) && config.optimizer.enable_dynamic_filter_pushdown { - let target_side = dynamic_filter_side(self.join_type); - let dynamic_filter = - Arc::clone(&self.dynamic_filter) as Arc; - match target_side { - JoinSide::Left => { - left_child = left_child.with_self_filter(dynamic_filter); - } - JoinSide::Right => { - right_child = right_child.with_self_filter(dynamic_filter); + if let Some(df) = &self.dynamic_filter { + let target_side = dynamic_filter_side(self.join_type); + let dynamic_filter = Arc::clone(df) as Arc; + match target_side { + JoinSide::Left => { + left_child = left_child.with_self_filter(dynamic_filter); + } + JoinSide::Right => { + right_child = right_child.with_self_filter(dynamic_filter); + } + JoinSide::None => {} } - JoinSide::None => {} } } @@ -1981,6 +1991,27 @@ mod tests { a.div_ceil(b) } + #[test] + fn dynamic_filter_side_truth_table() { + use JoinSide::{Left as SideLeft, None as SideNone, Right as SideRight}; + use JoinType::*; + let cases = [ + (Inner, SideRight), + (Left, SideRight), + (Right, SideLeft), + (Full, SideNone), + (LeftSemi, SideRight), + (LeftAnti, SideRight), + (RightSemi, SideLeft), + (RightAnti, SideLeft), + (LeftMark, SideRight), + (RightMark, SideLeft), + ]; + for (join_type, expected) in cases { + assert_eq!(dynamic_filter_side(join_type), expected, "{join_type:?}"); + } + } + #[template] #[rstest] fn batch_sizes(#[values(8192, 10, 5, 2, 1)] batch_size: usize) {} From e5bf48485634a06ee64c566017f4ad194ea9a798 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 17:44:40 +0800 Subject: [PATCH 17/72] Update explain_tree.slt to reflect changes in predicate representation for clarity --- datafusion/sqllogictest/test_files/explain_tree.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 550d06603b1a..e56d8f1a273f 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1241,7 +1241,7 @@ physical_plan 31)│ -------------------- ││ -------------------- │ 32)│ files: 1 ││ partition_count(in->out): │ 33)│ format: parquet ││ 1 -> 4 │ -34)│ predicate: true ││ │ +34)│ predicate: ││ │ 35)│ ││ partitioning_scheme: │ 36)│ ││ RoundRobinBatch(4) │ 37)└───────────────────────────┘└─────────────┬─────────────┘ From 1eba0d00c13c1e177548a0cb760e75b27d63eccd Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 17:47:20 +0800 Subject: [PATCH 18/72] Refactor dynamic filter pushdown documentation for clarity and completeness --- datafusion/sqllogictest/test_files/information_schema.slt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 68cb6f2eeca7..58c431cbc1b6 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -399,7 +399,8 @@ datafusion.format.types_info false Show types in visual representation batches datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. -datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attempts to push down dynamic filters generated by operators into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. Dynamic filters are also produced by left, right, semi, and anti joins, allowing DataFusion to prune the probe side during execution. Full joins are not supported. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. +datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attempts to push down dynamic filters generated by operators into the file scan phase. Dynamic filters are produced by left, right, semi, and anti joins, allowing DataFusion to prune the probe side during execution. + Full joins are not supported and only equi-join keys generate filters. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible datafusion.optimizer.expand_views_at_output false When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. From c48e8c10097727be65e1570c9d5c849b9c5aa38f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 17:47:40 +0800 Subject: [PATCH 19/72] Improve changelog entry for dynamic filter pushdown in version 50.0.0 with additional details on usage and limitations --- dev/changelog/50.0.0.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/changelog/50.0.0.md b/dev/changelog/50.0.0.md index 4ac23495d9cd..14e74d3280d9 100644 --- a/dev/changelog/50.0.0.md +++ b/dev/changelog/50.0.0.md @@ -21,4 +21,4 @@ under the License. This release is under active development. -- Enable dynamic filter pushdown for left, right, semi, and anti joins [#16445](https://github.com/apache/datafusion/pull/16445) (adriangb) +- Enable dynamic filter pushdown for left, right, semi, and anti joins [#16445](https://github.com/apache/datafusion/pull/16445) (adriangb). This non-breaking change is enabled by default and supports pruning the probe side of `Inner`, `Left`, `Right`, `LeftSemi`, `LeftAnti`, `RightSemi`, and `RightAnti` hash joins. Full joins and non-equi predicates are not yet supported. Dynamic filters add planning overhead for high-cardinality keys; disable via `datafusion.optimizer.enable_dynamic_filter_pushdown=false` if necessary. From 41f2e1e2bb90b586c54cd1e6df0d12bf436bc1eb Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 17:47:43 +0800 Subject: [PATCH 20/72] Enhance upgrading documentation to include details on dynamic filter pushdown behavior and configuration options. Added a table summarizing which join types support probe side pruning and clarified the functionality with respect to equi-join keys. --- docs/source/library-user-guide/upgrading.md | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/docs/source/library-user-guide/upgrading.md b/docs/source/library-user-guide/upgrading.md index 42c76f4369fd..b7e22982641f 100644 --- a/docs/source/library-user-guide/upgrading.md +++ b/docs/source/library-user-guide/upgrading.md @@ -44,10 +44,22 @@ returns a `ColumnarValue` instead of a `ArrayRef`. Dynamic filter pushdown now applies to left, right, semi and anti joins, allowing DataFusion to prune the probe side as join keys are discovered at -runtime. Full joins are not supported. This behavior is controlled by the +runtime. Full joins are not supported and only equi-join keys contribute to the +filters. This behavior is controlled by the `datafusion.optimizer.enable_dynamic_filter_pushdown` configuration option (on by default). +| JoinType | Probe side pruned | +|-----------------------------|------------------| +| `Inner`, `Left` | Right input | +| `Right` | Left input | +| `LeftSemi`, `LeftAnti` | Left input | +| `RightSemi`, `RightAnti` | Right input | + +Dynamic filters are most effective when the join keys are highly selective. +You can disable the feature by setting +`datafusion.optimizer.enable_dynamic_filter_pushdown=false`. + For example: ```sql From 4d507c42425895875267c3a6500a0f4069dc936e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 17:47:54 +0800 Subject: [PATCH 21/72] Update dynamic filter pushdown description for clarity and detail - Revised the explanation of the `datafusion.optimizer.enable_dynamic_filter_pushdown` configuration to clarify the behavior when filters are generated by various join types and to emphasize the limitations regarding full joins and equi-join keys. - Added a note to reference the upgrade guide for additional details. --- docs/source/user-guide/configs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 8c57502221fd..00ab995611f2 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -129,7 +129,7 @@ The following configuration settings are available: | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. Dynamic filters are also produced by left, right, semi, and anti joins, allowing DataFusion to prune the probe side during execution. Full joins are not supported. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. | +| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators into the file scan phase. Dynamic filters are produced by left, right, semi, and anti joins, pruning the probe side. Full joins are not supported and only equi-join keys generate filters. See the upgrade guide for details. Interacts with `execution.parquet.pushdown_filters`. | | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | | datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | From 25254ab7c0ea9111a23074857dc9a58ed2c64bf7 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 18:36:10 +0800 Subject: [PATCH 22/72] Enhance documentation for dynamic filter pushdown, clarifying join types and probe side mapping. Added details on predicate pushdown requirements and examples. --- datafusion/common/src/config.rs | 32 ++++++++++++++++++++++++-------- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index e0435f595b41..8c98786886ef 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -731,14 +731,30 @@ config_namespace! { /// to push down the current top 10 timestamps that the TopK operator references /// into the file scans. This means that if we already have 10 timestamps in the /// year 2025 any files that only have timestamps in the year 2024 can be skipped / - /// pruned at various stages in the scan. Dynamic filters are also produced by - /// left, right, semi, and anti joins, allowing DataFusion to prune the probe side - /// during execution. For hash joins, dynamic filters are applied to the **probe** - /// side (the stream that is scanned), which is the right input for `Inner` and - /// `Left` joins and the left input for `Right` joins; semi and anti joins prune the - /// left/probe input. Full joins are not supported and non-equi join predicates do - /// **not** generate dynamic filters. For example, - /// `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` + /// pruned at various stages in the scan. + /// + /// Dynamic filters are also produced by left, right, semi, and anti joins, + /// allowing DataFusion to prune the probe side during execution. The *probe + /// side* is the stream that is scanned in a hash join. For hash joins the + /// mapping is: + /// + /// | Join type | Probe side | + /// |-------------------------|-----------| + /// | `Inner`, `Left` | Right input | + /// | `Right` | Left input | + /// | `LeftSemi`, `LeftAnti` | Left input | + /// | `RightSemi`, `RightAnti`| Right input | + /// + /// Full joins are not supported. + /// + /// Non-equi join predicates do **not** generate dynamic filters as they require + /// range or conjunctive analysis; composite predicates only derive filters from + /// their equi-conjuncts. + /// + /// Pushdown is effective only when the file source supports predicate pushdown + /// (e.g. Parquet) and `execution.parquet.pushdown_filters` is enabled. + /// See the upgrade guide for additional details and examples. + /// For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` /// will only read `fact` rows whose `id` values match `dim` rows where /// `region = 'US'`. pub enable_dynamic_filter_pushdown: bool, default = true From 7ec25531467a79a31c75bcf55fae9529d6f36604 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 19:22:31 +0800 Subject: [PATCH 23/72] Update predicate display in TestSource to show '' for absent predicates --- .../physical_optimizer/filter_pushdown/mod.rs | 189 ++++++++++++++---- .../filter_pushdown/util.rs | 2 +- 2 files changed, 156 insertions(+), 35 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 9b11f7296040..207d9284756f 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -50,7 +50,7 @@ use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, coalesce_batches::CoalesceBatchesExec, filter::FilterExec, - joins::{HashJoinExec, PartitionMode}, + joins::{utils::JoinFilter, HashJoinExec, PartitionMode}, repartition::RepartitionExec, sorts::sort::SortExec, ExecutionPlan, @@ -369,8 +369,8 @@ async fn test_static_filter_pushdown_through_hash_join() { - FilterExec: e@4 = ba - FilterExec: a@0 = aa - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate= output: Ok: - FilterExec: a@0 = d@3 @@ -524,7 +524,7 @@ fn test_push_down_through_transparent_nodes() { - RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=1 - FilterExec: a@0 = foo - CoalesceBatchesExec: target_batch_size=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - RepartitionExec: partitioning=RoundRobinBatch(1), input_partitions=1 @@ -809,12 +809,12 @@ async fn test_hashjoin_dynamic_filter_pushdown() { OptimizationTest: input: - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate= output: Ok: - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] ", ); @@ -843,7 +843,7 @@ async fn test_hashjoin_dynamic_filter_pushdown() { format!("{}", format_plan_for_test(&plan)), @r" - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], filter=[a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb ] " ); @@ -1086,8 +1086,8 @@ async fn test_hashjoin_parent_filter_pushdown() { - FilterExec: e@4 = ba - FilterExec: a@0 = aa - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate= output: Ok: - FilterExec: a@0 = d@3 @@ -1102,6 +1102,7 @@ fn build_join_with_dynamic_filter( join_type: JoinType, left_support: bool, right_support: bool, + partition_mode: PartitionMode, ) -> Arc { let left_batches = vec![record_batch!( ("a", Utf8, ["aa", "ab"]), @@ -1154,16 +1155,65 @@ fn build_join_with_dynamic_filter( None, &join_type, None, - PartitionMode::Partitioned, + partition_mode, datafusion_common::NullEquality::NullEqualsNothing, ) .unwrap(), ) as Arc } +#[tokio::test] +async fn test_hashjoin_non_equi_predicate_no_dynamic_filter() { + // Non-equi join predicates like `l.a > r.a` are not supported by HashJoinExec + // and thus cannot produce dynamic filters. + let left_batches = vec![record_batch!(("a", Utf8, ["aa", "ab"])).unwrap()]; + let left_schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)])); + let left_scan = TestScanBuilder::new(Arc::clone(&left_schema)) + .with_support(true) + .with_batches(left_batches) + .build(); + + let right_batches = vec![record_batch!(("a", Utf8, ["aa", "ab"])).unwrap()]; + let right_schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)])); + let right_scan = TestScanBuilder::new(Arc::clone(&right_schema)) + .with_support(true) + .with_batches(right_batches) + .build(); + + let filter_expr = Arc::new(BinaryExpr::new( + col("a", &left_schema).unwrap(), + Operator::Gt, + col("a", &right_schema).unwrap(), + )) as Arc; + let column_indices = JoinFilter::build_column_indices(vec![0], vec![0]); + let filter_schema = Arc::new(Schema::new(vec![ + left_schema.field(0).clone(), + right_schema.field(0).clone(), + ])); + let join_filter = JoinFilter::new(filter_expr, column_indices, filter_schema); + + let res = HashJoinExec::try_new( + left_scan, + right_scan, + vec![], + Some(join_filter), + &JoinType::Inner, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ); + assert!(res.is_err(), "non-equi joins should be rejected"); +} + #[tokio::test] async fn test_hashjoin_left_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::Left, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::Left, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1175,15 +1225,20 @@ async fn test_hashjoin_left_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); - assert_not_contains!( + assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate" + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); } #[tokio::test] async fn test_hashjoin_left_dynamic_filter_pushdown_disabled() { - let plan = build_join_with_dynamic_filter(JoinType::Left, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::Left, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = false; config.optimizer.enable_dynamic_filter_pushdown = false; @@ -1196,7 +1251,12 @@ async fn test_hashjoin_left_dynamic_filter_pushdown_disabled() { #[tokio::test] async fn test_hashjoin_right_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::Right, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::Right, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1208,15 +1268,20 @@ async fn test_hashjoin_right_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); - assert_not_contains!( + assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); } #[tokio::test] async fn test_hashjoin_right_dynamic_filter_pushdown_disabled() { - let plan = build_join_with_dynamic_filter(JoinType::Right, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::Right, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = false; config.optimizer.enable_dynamic_filter_pushdown = false; @@ -1227,9 +1292,39 @@ async fn test_hashjoin_right_dynamic_filter_pushdown_disabled() { assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); } +#[tokio::test] +async fn test_hashjoin_left_dynamic_filter_pushdown_collect_left() { + let plan = build_join_with_dynamic_filter( + JoinType::Left, + true, + true, + PartitionMode::CollectLeft, + ); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" + ); +} + #[tokio::test] async fn test_hashjoin_left_semi_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::LeftSemi, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::LeftSemi, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1241,15 +1336,20 @@ async fn test_hashjoin_left_semi_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); - assert_not_contains!( + assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate" + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); } #[tokio::test] async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::LeftAnti, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::LeftAnti, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1261,15 +1361,20 @@ async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); - assert_not_contains!( + assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate" + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); } #[tokio::test] async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::RightSemi, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::RightSemi, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1281,15 +1386,20 @@ async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); - assert_not_contains!( + assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); } #[tokio::test] async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::RightAnti, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::RightAnti, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1301,15 +1411,20 @@ async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); - assert_not_contains!( + assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); } #[tokio::test] async fn test_hashjoin_full_dynamic_filter_no_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::Full, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::Full, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1322,7 +1437,12 @@ async fn test_hashjoin_full_dynamic_filter_no_pushdown() { #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown_unsupported() { - let plan = build_join_with_dynamic_filter(JoinType::Left, true, false); + let plan = build_join_with_dynamic_filter( + JoinType::Left, + true, + false, + PartitionMode::CollectLeft, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1335,7 +1455,8 @@ async fn test_hashjoin_dynamic_filter_pushdown_unsupported() { #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown_unsupported_left() { - let plan = build_join_with_dynamic_filter(JoinType::Right, false, true); + let plan = + build_join_with_dynamic_filter(JoinType::Right, false, true, PartitionMode::Auto); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index acb2b808ef8f..74239f5f0b23 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -199,7 +199,7 @@ impl FileSource for TestSource { .predicate .as_ref() .map(|p| format!(", predicate={p}")) - .unwrap_or_default(); + .unwrap_or_else(|| ", predicate=".to_string()); write!(f, "{support}{predicate_string}") } From 25b4dfb5e76db84df14561bade21d4d13b5b6ec9 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 19:49:10 +0800 Subject: [PATCH 24/72] Add tests for dynamic filter pushdown in HashJoinExec, including null keys and high cardinality scenarios --- .../physical_optimizer/filter_pushdown/mod.rs | 192 ++++++++++++++++++ 1 file changed, 192 insertions(+) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 207d9284756f..cbd17ee5a494 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -847,6 +847,182 @@ async fn test_hashjoin_dynamic_filter_pushdown() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb ] " ); + + let join = plan.as_any().downcast_ref::().unwrap(); + assert_projection(&join.left, &["a", "b", "c"]); + assert_projection(&join.right, &["a", "b", "e"]); +} + +#[tokio::test] +async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { + use datafusion_common::JoinType; + use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; + + // Build side containing only null join keys + let build_batches = vec![ + record_batch!( + ("a", Utf8, [None::<&str>, None]), + ("b", Utf8, [None::<&str>, None]), + ("c", Float64, [Some(1.0), Some(2.0)]) + ) + .unwrap(), + ]; + let build_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, true), + Field::new("b", DataType::Utf8, true), + Field::new("c", DataType::Float64, true), + ])); + let build_scan = TestScanBuilder::new(Arc::clone(&build_schema)) + .with_support(true) + .with_batches(build_batches) + .build(); + + // Probe side with regular values + let probe_batches = vec![ + record_batch!(("a", Utf8, ["aa", "ab", "ac"]), ("b", Utf8, ["ba", "bb", "bc"]), ("e", Float64, [1.0, 2.0, 3.0])) + .unwrap(), + ]; + let probe_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, false), + Field::new("b", DataType::Utf8, false), + Field::new("e", DataType::Float64, false), + ])); + let probe_scan = TestScanBuilder::new(Arc::clone(&probe_schema)) + .with_support(true) + .with_batches(probe_batches) + .build(); + + let on = vec![ + ( + col("a", &build_schema).unwrap(), + col("a", &probe_schema).unwrap(), + ), + ( + col("b", &build_schema).unwrap(), + col("b", &probe_schema).unwrap(), + ), + ]; + let plan = Arc::new( + HashJoinExec::try_new( + build_scan, + probe_scan, + on, + None, + &JoinType::Inner, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ) + .unwrap(), + ) as Arc; + + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let session_cfg = SessionConfig::new().with_batch_size(10); + let session_ctx = SessionContext::new_with_config(session_cfg); + session_ctx.register_object_store( + ObjectStoreUrl::parse("test://").unwrap().as_ref(), + Arc::new(InMemory::new()), + ); + let state = session_ctx.state(); + let task_ctx = state.task_ctx(); + let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); + stream.next().await.unwrap().unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + formatted, + "DynamicFilterPhysicalExpr [ a@0 >= NULL AND a@0 <= NULL AND b@1 >= NULL AND b@1 <= NULL ]" + ); + + let join = plan.as_any().downcast_ref::().unwrap(); + assert_projection(&join.left, &["a", "b", "c"]); + assert_projection(&join.right, &["a", "b", "e"]); +} + +#[tokio::test] +async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { + use datafusion_common::JoinType; + use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; + use arrow::array::{ArrayRef, Float64Array, Int32Array}; + use arrow::record_batch::RecordBatch; + + // Generate large key sets to watch for planning regressions + let size = 10_000; + let build_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("x", DataType::Float64, false), + ])); + let build_a: Vec> = (0..size).map(Some).collect(); + let build_x: Vec> = (0..size).map(|v| Some(v as f64)).collect(); + let build_batch = RecordBatch::try_new( + Arc::clone(&build_schema), + vec![ + Arc::new(Int32Array::from(build_a.clone())) as ArrayRef, + Arc::new(Float64Array::from(build_x.clone())) as ArrayRef, + ], + ) + .unwrap(); + let build_batches = vec![build_batch]; + let build_scan = TestScanBuilder::new(Arc::clone(&build_schema)) + .with_support(true) + .with_batches(build_batches) + .build(); + + let probe_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("y", DataType::Float64, false), + ])); + let probe_a: Vec> = (0..size).map(Some).collect(); + let probe_y: Vec> = (0..size).map(|v| Some(v as f64)).collect(); + let probe_batch = RecordBatch::try_new( + Arc::clone(&probe_schema), + vec![ + Arc::new(Int32Array::from(probe_a.clone())) as ArrayRef, + Arc::new(Float64Array::from(probe_y.clone())) as ArrayRef, + ], + ) + .unwrap(); + let probe_batches = vec![probe_batch]; + let probe_scan = TestScanBuilder::new(Arc::clone(&probe_schema)) + .with_support(true) + .with_batches(probe_batches) + .build(); + + // High-cardinality join keys may increase planning time but should still allow dynamic filter pushdown + let on = vec![( + col("a", &build_schema).unwrap(), + col("a", &probe_schema).unwrap(), + )]; + let plan = Arc::new( + HashJoinExec::try_new( + build_scan, + probe_scan, + on, + None, + &JoinType::Inner, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ) + .unwrap(), + ) as Arc; + + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!(formatted, "DynamicFilterPhysicalExpr"); + + let join = plan.as_any().downcast_ref::().unwrap(); + assert_projection(&join.left, &["a", "x"]); + assert_projection(&join.right, &["a", "y"]); } #[tokio::test] @@ -995,6 +1171,12 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= ca AND d@0 <= ce ] " ); + + let outer = plan.as_any().downcast_ref::().unwrap(); + assert_projection(&outer.left, &["a", "x"]); + let inner = outer.right.as_any().downcast_ref::().unwrap(); + assert_projection(&inner.left, &["b", "c", "y"]); + assert_projection(&inner.right, &["d", "z"]); } #[tokio::test] @@ -1523,6 +1705,16 @@ STORED AS PARQUET; // Pushdown pruned most rows } +fn assert_projection(plan: &Arc, expected: &[&str]) { + let schema = plan.schema(); + let actual: Vec<_> = schema + .fields() + .iter() + .map(|f| f.name().as_str()) + .collect(); + assert_eq!(actual, expected); +} + /// Schema: /// a: String /// b: String From 6d5b6ce7364c794042010288a4e344586b7742df Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 19:49:54 +0800 Subject: [PATCH 25/72] Update dynamic filter pushdown documentation and configuration description to clarify behavior and requirements for join types and file formats. --- docs/source/library-user-guide/upgrading.md | 27 +++++++++++++-------- docs/source/user-guide/configs.md | 2 +- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/docs/source/library-user-guide/upgrading.md b/docs/source/library-user-guide/upgrading.md index b7e22982641f..3ace9dbf502d 100644 --- a/docs/source/library-user-guide/upgrading.md +++ b/docs/source/library-user-guide/upgrading.md @@ -45,16 +45,22 @@ returns a `ColumnarValue` instead of a `ArrayRef`. Dynamic filter pushdown now applies to left, right, semi and anti joins, allowing DataFusion to prune the probe side as join keys are discovered at runtime. Full joins are not supported and only equi-join keys contribute to the -filters. This behavior is controlled by the -`datafusion.optimizer.enable_dynamic_filter_pushdown` configuration option (on by -default). - -| JoinType | Probe side pruned | -|-----------------------------|------------------| -| `Inner`, `Left` | Right input | -| `Right` | Left input | -| `LeftSemi`, `LeftAnti` | Left input | -| `RightSemi`, `RightAnti` | Right input | +filters. Dynamic filters obey `NullEqualsNothing` semantics, so rows with null +join keys never match and generate no filter values. This behavior is controlled +by the `datafusion.optimizer.enable_dynamic_filter_pushdown` configuration +option (on by default). Dynamic filter pushdown requires file formats that +support predicate pushdown; for Parquet this means enabling +`datafusion.execution.parquet.pushdown_filters`. + +In a hash join, the probe side is the input whose rows are scanned to find +matches against the hashed build side. + +| JoinType | Probe side pruned | +| ------------------------ | ----------------- | +| `Inner`, `Left` | Right input | +| `Right` | Left input | +| `LeftSemi`, `LeftAnti` | Left input | +| `RightSemi`, `RightAnti` | Right input | Dynamic filters are most effective when the join keys are highly selective. You can disable the feature by setting @@ -71,6 +77,7 @@ WHERE dim.region = 'US'; As rows from `dim` with `region = 'US'` are processed, a dynamic filter is generated that skips `fact` partitions without matching `id` values. +Plan effect: the `fact` scan receives `DynamicFilter{fact.id}`. To upgrade, change the return type of your implementation diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 00ab995611f2..11b578ac2d08 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -129,7 +129,7 @@ The following configuration settings are available: | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators into the file scan phase. Dynamic filters are produced by left, right, semi, and anti joins, pruning the probe side. Full joins are not supported and only equi-join keys generate filters. See the upgrade guide for details. Interacts with `execution.parquet.pushdown_filters`. | +| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by left, right, semi, and anti joins into the file scan phase, pruning the probe side. Full joins are not supported and only equi-join keys generate filters. See the [join-type table](../library-user-guide/upgrading.md#dynamic-filter-pushdown-for-joins) for details. Disable for high-cardinality keys or file formats without predicate pushdown (e.g., CSV/JSON). Interacts with `execution.parquet.pushdown_filters`. | | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | | datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | From 9c2ba0b5d1e41ef6bd8e8adde321296c0674f316 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 19:50:25 +0800 Subject: [PATCH 26/72] Enhance dynamic filter pushdown documentation to include support for mark joins and provide examples of usage in star-schema workloads. --- dev/changelog/50.0.0.md | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/dev/changelog/50.0.0.md b/dev/changelog/50.0.0.md index 14e74d3280d9..c1e8926614fa 100644 --- a/dev/changelog/50.0.0.md +++ b/dev/changelog/50.0.0.md @@ -21,4 +21,14 @@ under the License. This release is under active development. -- Enable dynamic filter pushdown for left, right, semi, and anti joins [#16445](https://github.com/apache/datafusion/pull/16445) (adriangb). This non-breaking change is enabled by default and supports pruning the probe side of `Inner`, `Left`, `Right`, `LeftSemi`, `LeftAnti`, `RightSemi`, and `RightAnti` hash joins. Full joins and non-equi predicates are not yet supported. Dynamic filters add planning overhead for high-cardinality keys; disable via `datafusion.optimizer.enable_dynamic_filter_pushdown=false` if necessary. +- Enable dynamic filter pushdown for left, right, semi, anti, and mark joins [#16445](https://github.com/apache/datafusion/pull/16445) (adriangb). + This non-breaking change is enabled by default and supports pruning the probe side of `Inner`, `Left`, `Right`, `LeftSemi`, `LeftAnti`, `RightSemi`, `RightAnti`, `LeftMark`, and `RightMark` hash joins. + For example, in star-schema workloads joining `dim` tables to a large `fact` table, dynamic filters can drop entire partitions of the `fact` table before they are scanned. + Full joins and non‑equi (range or composite) predicates are not yet supported; see [#7955](https://github.com/apache/datafusion/issues/7955). + Dynamic filters add planning overhead for high-cardinality keys; disable via the environment variable: + + ```bash + export DATAFUSION_OPTIMIZER_ENABLE_DYNAMIC_FILTER_PUSHDOWN=0 + ``` + + or by setting `datafusion.optimizer.enable_dynamic_filter_pushdown=false` programmatically if necessary. From 051c3109a5013aa5b8b8d46d358b3e6470b60abf Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 19:52:24 +0800 Subject: [PATCH 27/72] Update predicate display in explain_tree.slt to show 'true' instead of '' --- datafusion/sqllogictest/test_files/explain_tree.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index e56d8f1a273f..550d06603b1a 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1241,7 +1241,7 @@ physical_plan 31)│ -------------------- ││ -------------------- │ 32)│ files: 1 ││ partition_count(in->out): │ 33)│ format: parquet ││ 1 -> 4 │ -34)│ predicate: ││ │ +34)│ predicate: true ││ │ 35)│ ││ partitioning_scheme: │ 36)│ ││ RoundRobinBatch(4) │ 37)└───────────────────────────┘└─────────────┬─────────────┘ From feeb231ef0e8c35d0a9c81ca495de3c1e6da19a0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 20:05:43 +0800 Subject: [PATCH 28/72] Update dynamic filter pushdown description to clarify requirements for execution.parquet.pushdown_filters --- datafusion/sqllogictest/test_files/information_schema.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 58c431cbc1b6..37b32de93634 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -399,7 +399,7 @@ datafusion.format.types_info false Show types in visual representation batches datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. -datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attempts to push down dynamic filters generated by operators into the file scan phase. Dynamic filters are produced by left, right, semi, and anti joins, allowing DataFusion to prune the probe side during execution. +datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attempts to push down dynamic filters generated by operators into the file scan phase. Dynamic filters are produced by left, right, semi, and anti joins, allowing DataFusion to prune the probe side during execution (requires `execution.parquet.pushdown_filters` to be true). Full joins are not supported and only equi-join keys generate filters. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible From f9c3cd05b8f6278737436769fced3fb76a2c7024 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 20:45:10 +0800 Subject: [PATCH 29/72] Enhance dynamic filter functionality by adding key count tracking and updating logic across HashJoinExec, SortExec, and TopK. Update related tests to reflect changes in dynamic filter behavior. --- .../physical_optimizer/filter_pushdown/mod.rs | 50 +++++++++---------- .../src/expressions/dynamic_filters.rs | 27 ++++++++-- .../physical-plan/src/joins/hash_join.rs | 19 ++++--- datafusion/physical-plan/src/sorts/sort.rs | 30 ++++++++++- datafusion/physical-plan/src/topk/mod.rs | 12 +++-- 5 files changed, 95 insertions(+), 43 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index cbd17ee5a494..952bfc56a7b7 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -245,8 +245,8 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { insta::assert_snapshot!( format_plan_for_test(&plan), @r" - - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false] - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] + - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter_keys=0 + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], filter_target=Right, filter_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] " @@ -268,8 +268,8 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { insta::assert_snapshot!( format_plan_for_test(&plan), @r" - - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter=[e@4 IS NULL OR e@4 < bb] - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], filter=[d@0 >= aa AND d@0 <= ab] + - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter=[e@4 IS NULL OR e@4 < bb], filter_keys=2 + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], filter=[d@0 >= aa AND d@0 <= ab], filter_target=Right, filter_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= aa AND d@0 <= ab ] AND DynamicFilterPhysicalExpr [ e@1 IS NULL OR e@1 < bb ] " @@ -732,7 +732,7 @@ async fn test_topk_dynamic_filter_pushdown() { insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" - - SortExec: TopK(fetch=1), expr=[b@1 DESC NULLS LAST], preserve_partitioning=[false], filter=[b@1 > bd] + - SortExec: TopK(fetch=1), expr=[b@1 DESC NULLS LAST], preserve_partitioning=[false], filter=[b@1 > bd], filter_keys=1 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ b@1 > bd ] " ); @@ -842,7 +842,7 @@ async fn test_hashjoin_dynamic_filter_pushdown() { insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], filter=[a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], filter=[a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb], filter_target=Right, filter_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb ] " @@ -859,14 +859,12 @@ async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; // Build side containing only null join keys - let build_batches = vec![ - record_batch!( - ("a", Utf8, [None::<&str>, None]), - ("b", Utf8, [None::<&str>, None]), - ("c", Float64, [Some(1.0), Some(2.0)]) - ) - .unwrap(), - ]; + let build_batches = vec![record_batch!( + ("a", Utf8, [None::<&str>, None]), + ("b", Utf8, [None::<&str>, None]), + ("c", Float64, [Some(1.0), Some(2.0)]) + ) + .unwrap()]; let build_schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Utf8, true), Field::new("b", DataType::Utf8, true), @@ -878,10 +876,12 @@ async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { .build(); // Probe side with regular values - let probe_batches = vec![ - record_batch!(("a", Utf8, ["aa", "ab", "ac"]), ("b", Utf8, ["ba", "bb", "bc"]), ("e", Float64, [1.0, 2.0, 3.0])) - .unwrap(), - ]; + let probe_batches = vec![record_batch!( + ("a", Utf8, ["aa", "ab", "ac"]), + ("b", Utf8, ["ba", "bb", "bc"]), + ("e", Float64, [1.0, 2.0, 3.0]) + ) + .unwrap()]; let probe_schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Utf8, false), Field::new("b", DataType::Utf8, false), @@ -945,10 +945,10 @@ async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; use arrow::array::{ArrayRef, Float64Array, Int32Array}; use arrow::record_batch::RecordBatch; + use datafusion_common::JoinType; + use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; // Generate large key sets to watch for planning regressions let size = 10_000; @@ -1164,9 +1164,9 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], filter=[b@0 >= aa AND b@0 <= ab] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], filter=[b@0 >= aa AND b@0 <= ab], filter_target=Right, filter_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], filter=[d@0 >= ca AND d@0 <= ce] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], filter=[d@0 >= ca AND d@0 <= ce], filter_target=Right, filter_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ b@0 >= aa AND b@0 <= ab ] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= ca AND d@0 <= ce ] " @@ -1707,11 +1707,7 @@ STORED AS PARQUET; fn assert_projection(plan: &Arc, expected: &[&str]) { let schema = plan.schema(); - let actual: Vec<_> = schema - .fields() - .iter() - .map(|f| f.name().as_str()) - .collect(); + let actual: Vec<_> = schema.fields().iter().map(|f| f.name().as_str()).collect(); assert_eq!(actual, expected); } diff --git a/datafusion/physical-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index d4b3180a6fc6..00a5540b3010 100644 --- a/datafusion/physical-expr/src/expressions/dynamic_filters.rs +++ b/datafusion/physical-expr/src/expressions/dynamic_filters.rs @@ -19,7 +19,10 @@ use std::{ any::Any, fmt::Display, hash::Hash, - sync::{Arc, RwLock}, + sync::{ + atomic::{AtomicUsize, Ordering}, + Arc, RwLock, + }, }; use crate::PhysicalExpr; @@ -48,6 +51,8 @@ pub struct DynamicFilterPhysicalExpr { remapped_children: Option>>, /// The source of dynamic filters. inner: Arc>, + /// Number of keys currently contained in this dynamic filter. + key_count: Arc, /// For testing purposes track the data type and nullability to make sure they don't change. /// If they do, there's a bug in the implementation. /// But this can have overhead in production, so it's only included in our tests. @@ -138,6 +143,7 @@ impl DynamicFilterPhysicalExpr { children, remapped_children: None, // Initially no remapped children inner: Arc::new(RwLock::new(Inner::new(inner))), + key_count: Arc::new(AtomicUsize::new(0)), data_type: Arc::new(RwLock::new(None)), nullable: Arc::new(RwLock::new(None)), } @@ -198,7 +204,11 @@ impl DynamicFilterPhysicalExpr { /// This should be called e.g.: /// - When we've computed the probe side's hash table in a HashJoinExec /// - After every batch is processed if we update the TopK heap in a SortExec using a TopK approach. - pub fn update(&self, new_expr: Arc) -> Result<()> { + pub fn update( + &self, + new_expr: Arc, + key_count: usize, + ) -> Result<()> { let mut current = self.inner.write().map_err(|_| { datafusion_common::DataFusionError::Execution( "Failed to acquire write lock for inner".to_string(), @@ -217,8 +227,14 @@ impl DynamicFilterPhysicalExpr { current.expr = new_expr; // Increment the generation to indicate that the expression has changed. current.generation += 1; + self.key_count.store(key_count, Ordering::SeqCst); Ok(()) } + + /// Return the current number of keys represented by this dynamic filter. + pub fn key_count(&self) -> usize { + self.key_count.load(Ordering::SeqCst) + } } impl PhysicalExpr for DynamicFilterPhysicalExpr { @@ -242,6 +258,7 @@ impl PhysicalExpr for DynamicFilterPhysicalExpr { children: self.children.clone(), remapped_children: Some(children), inner: Arc::clone(&self.inner), + key_count: Arc::clone(&self.key_count), data_type: Arc::clone(&self.data_type), nullable: Arc::clone(&self.nullable), })) @@ -433,7 +450,7 @@ mod test { lit(43) as Arc, )); dynamic_filter - .update(Arc::clone(&new_expr) as Arc) + .update(Arc::clone(&new_expr) as Arc, 0) .expect("Failed to update expression"); // Now we should be able to evaluate the new expression on both batches let result_1 = dynamic_filter_1.evaluate(&batch_1).unwrap(); @@ -463,7 +480,7 @@ mod test { // Update the current expression let new_expr = lit(100) as Arc; - dynamic_filter.update(Arc::clone(&new_expr)).unwrap(); + dynamic_filter.update(Arc::clone(&new_expr), 0).unwrap(); // Take another snapshot let snapshot = dynamic_filter.snapshot().unwrap(); assert_eq!(snapshot, Some(new_expr)); @@ -492,7 +509,7 @@ mod test { // Now change the current expression to something else. dynamic_filter - .update(lit(ScalarValue::Utf8(None)) as Arc) + .update(lit(ScalarValue::Utf8(None)) as Arc, 0) .expect("Failed to update expression"); // Check that we error if we call data_type, nullable or evaluate after changing the expression. assert!( diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 931cea72cbf2..7d618bf17adc 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -745,12 +745,19 @@ impl DisplayAs for HashJoinExec { .collect::>() .join(", "); let dynamic_filter_display = match &self.dynamic_filter { - Some(df) => match df.current() { - Ok(current) if current != lit(true) => { - format!(", filter=[{current}]") + Some(df) => { + let target = dynamic_filter_side(self.join_type); + let keys = df.key_count(); + match df.current() { + Ok(current) if current != lit(true) => { + format!(", filter=[{current}], filter_target={:?}, filter_keys={}", target, keys) + } + _ => format!( + ", filter_target={:?}, filter_keys={}", + target, keys + ), } - _ => "".to_string(), - }, + } None => "".to_string(), }; write!( @@ -1318,7 +1325,7 @@ async fn collect_left_input( }) .unwrap_or_else(|| lit(true)); - dynamic_filter.update(combined_predicate)?; + dynamic_filter.update(combined_predicate, num_rows)?; } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index dc2a5640f40b..dd798585f25c 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1051,10 +1051,18 @@ impl DisplayAs for SortExec { Some(fetch) => { write!(f, "SortExec: TopK(fetch={fetch}), expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr)?; if let Some(filter) = &self.filter { + let keys = filter.key_count(); if let Ok(current) = filter.current() { if !current.eq(&lit(true)) { - write!(f, ", filter=[{current}]")?; + write!( + f, + ", filter=[{current}], filter_keys={keys}" + )?; + } else { + write!(f, ", filter_keys={keys}")?; } + } else { + write!(f, ", filter_keys={keys}")?; } } if !self.common_sort_prefix.is_empty() { @@ -1073,7 +1081,25 @@ impl DisplayAs for SortExec { Ok(()) } } - None => write!(f, "SortExec: expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr), + None => { + write!(f, "SortExec: expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr)?; + if let Some(filter) = &self.filter { + let keys = filter.key_count(); + if let Ok(current) = filter.current() { + if !current.eq(&lit(true)) { + write!( + f, + ", filter=[{current}], filter_keys={keys}" + )?; + } else { + write!(f, ", filter_keys={keys}")?; + } + } else { + write!(f, ", filter_keys={keys}")?; + } + } + Ok(()) + } } } DisplayFormatType::TreeRender => match self.fetch { diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 8d06fa73ce8e..d3bdac2384a7 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -404,9 +404,10 @@ impl TopK { .reduce(|a, b| Arc::new(BinaryExpr::new(a, Operator::Or, b))); if let Some(predicate) = dynamic_predicate { - if !predicate.eq(&lit(true)) { - filter.update(predicate)?; - } + filter.update(predicate, self.heap.len())?; + } else { + // even if predicate is true, record current key count + filter.update(lit(true), self.heap.len())?; } Ok(()) @@ -610,6 +611,11 @@ impl TopKHeap { } } + /// Return number of rows currently stored in the heap. + fn len(&self) -> usize { + self.inner.len() + } + /// Adds `row` to this heap. If inserting this new item would /// increase the size past `k`, removes the previously smallest /// item. From 47c24916d8a2bba996ca1f24d01fb5e0bc75d12e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 21:00:54 +0800 Subject: [PATCH 30/72] Refactor dynamic filter handling in HashJoinExec to clarify probe side and filter logic. Update comments for synchronization with optimizer's push_down_filter module. --- .../physical_optimizer/filter_pushdown/mod.rs | 60 +++++++++++++++++-- .../physical-plan/src/joins/hash_join.rs | 28 ++++++--- 2 files changed, 74 insertions(+), 14 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 952bfc56a7b7..221e3a1ceff6 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -246,7 +246,7 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { format_plan_for_test(&plan), @r" - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter_keys=0 - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], filter_target=Right, filter_keys=0 + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] " @@ -269,7 +269,7 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { format_plan_for_test(&plan), @r" - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter=[e@4 IS NULL OR e@4 < bb], filter_keys=2 - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], filter=[d@0 >= aa AND d@0 <= ab], filter_target=Right, filter_keys=2 + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_filter=[d@0 >= aa AND d@0 <= ab], probe_side=Right, probe_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= aa AND d@0 <= ab ] AND DynamicFilterPhysicalExpr [ e@1 IS NULL OR e@1 < bb ] " @@ -842,7 +842,7 @@ async fn test_hashjoin_dynamic_filter_pushdown() { insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], filter=[a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb], filter_target=Right, filter_keys=2 + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], probe_filter=[a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb], probe_side=Right, probe_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb ] " @@ -1164,9 +1164,9 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], filter=[b@0 >= aa AND b@0 <= ab], filter_target=Right, filter_keys=2 + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], probe_filter=[b@0 >= aa AND b@0 <= ab], probe_side=Right, probe_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], filter=[d@0 >= ca AND d@0 <= ce], filter_target=Right, filter_keys=2 + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_filter=[d@0 >= ca AND d@0 <= ce], probe_side=Right, probe_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ b@0 >= aa AND b@0 <= ab ] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= ca AND d@0 <= ce ] " @@ -1599,6 +1599,56 @@ async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { ); } +#[tokio::test] +async fn test_hashjoin_left_mark_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter( + JoinType::LeftMark, + true, + true, + PartitionMode::Partitioned, + ); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr", + ); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=", + ); +} + +#[tokio::test] +async fn test_hashjoin_right_mark_dynamic_filter_pushdown() { + let plan = build_join_with_dynamic_filter( + JoinType::RightMark, + true, + true, + PartitionMode::Partitioned, + ); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr", + ); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=", + ); +} + #[tokio::test] async fn test_hashjoin_full_dynamic_filter_no_pushdown() { let plan = build_join_with_dynamic_filter( diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 7d618bf17adc..51e5f6bd9858 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -94,7 +94,8 @@ use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::{ready, Stream, StreamExt, TryStreamExt}; use parking_lot::Mutex; -// Helper functions copied from optimizer's push_down_filter module +// Helper functions mirrored from the optimizer's push_down_filter module. +// KEEP IN SYNC WITH datafusion/optimizer/src/push_down_filter.rs // Determine which sides of a JOIN preserve rows for join output filters fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { match join_type { @@ -122,9 +123,10 @@ fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { } } -// Determine which side should receive the dynamic filter. -// Mark joins filter the opposite side of the preserved input to -// restrict rows feeding the ON clause evaluation. +/// Returns which side of the join should receive a dynamic filter. +/// +/// Mark joins apply filters to the *opposite* side of the preserved input so +/// that only rows capable of satisfying the ON clause are evaluated. fn dynamic_filter_side(join_type: JoinType) -> JoinSide { let (left_preserved, right_preserved) = lr_is_preserved(join_type); let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join_type); @@ -746,15 +748,18 @@ impl DisplayAs for HashJoinExec { .join(", "); let dynamic_filter_display = match &self.dynamic_filter { Some(df) => { - let target = dynamic_filter_side(self.join_type); + let probe_side = dynamic_filter_side(self.join_type); let keys = df.key_count(); match df.current() { Ok(current) if current != lit(true) => { - format!(", filter=[{current}], filter_target={:?}, filter_keys={}", target, keys) + format!( + ", probe_filter=[{current}], probe_side={:?}, probe_keys={}", + probe_side, keys + ) } _ => format!( - ", filter_target={:?}, filter_keys={}", - target, keys + ", probe_side={:?}, probe_keys={}", + probe_side, keys ), } } @@ -865,7 +870,10 @@ impl ExecutionPlan for HashJoinExec { self.mode, self.null_equality, )?; - // Preserve the dynamic filter if it exists + // Preserve the dynamic filter if it exists. + // The `on` clause is unchanged so the filter keys remain valid; if the + // `on` clause changes, `try_new` recomputes the keys via + // `create_dynamic_filter` to avoid stale filters. new_join.dynamic_filter = self.dynamic_filter.clone(); Ok(Arc::new(new_join)) } @@ -887,6 +895,8 @@ impl ExecutionPlan for HashJoinExec { column_indices: self.column_indices.clone(), null_equality: self.null_equality, cache: self.cache.clone(), + // Recompute dynamic filter to ensure keys reflect the current `on` + // expressions and avoid carrying stale filters across executions. dynamic_filter: Self::create_dynamic_filter(&self.on, self.join_type), })) } From 2c3209849c8a1598a2ce19a14dbd092ceb87dbaa Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 21:24:59 +0800 Subject: [PATCH 31/72] Enforce non-empty ON clause requirement in HashJoinExec and enhance dynamic filter handling. Add tests for join type preservation logic. --- .../physical-plan/src/joins/hash_join.rs | 76 ++++++++++++++++--- 1 file changed, 64 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 51e5f6bd9858..61290d483443 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -438,9 +438,10 @@ impl HashJoinExec { ) -> Result { let left_schema = left.schema(); let right_schema = right.schema(); - if on.is_empty() { - return plan_err!("On constraints in HashJoinExec should be non-empty"); - } + assert!( + !on.is_empty(), + "HashJoinExec requires a non-empty ON clause; empty lists are unsupported" + ); check_join_is_valid(&left_schema, &right_schema, &on)?; @@ -1098,6 +1099,7 @@ impl ExecutionPlan for HashJoinExec { config: &ConfigOptions, ) -> Result { let (left_preserved, right_preserved) = lr_is_preserved(self.join_type); + let dynamic_target = dynamic_filter_side(self.join_type); let unsupported: Vec<_> = parent_filters .iter() @@ -1106,11 +1108,19 @@ impl ExecutionPlan for HashJoinExec { }) .collect(); - let mut left_child = if left_preserved { - crate::filter_pushdown::ChildFilterDescription::from_child( + let mut left_child = if left_preserved || matches!(dynamic_target, JoinSide::Left) + { + let mut desc = crate::filter_pushdown::ChildFilterDescription::from_child( &parent_filters, self.left(), - )? + )?; + if !left_preserved { + // For joins like `RightAnti`, the left input is not preserved and + // parent filters can't be enforced. Mark them as unsupported so + // they will not be propagated further down. + desc.parent_filters = unsupported.clone(); + } + desc } else { crate::filter_pushdown::ChildFilterDescription { parent_filters: unsupported.clone(), @@ -1118,11 +1128,20 @@ impl ExecutionPlan for HashJoinExec { } }; - let mut right_child = if right_preserved { - crate::filter_pushdown::ChildFilterDescription::from_child( + let mut right_child = if right_preserved + || matches!(dynamic_target, JoinSide::Right) + { + let mut desc = crate::filter_pushdown::ChildFilterDescription::from_child( &parent_filters, self.right(), - )? + )?; + if !right_preserved { + // A `LeftAnti` join discards all rows from the right side, so + // any parent filter referencing it would be meaningless. Mark + // such filters unsupported to avoid incorrect pushdown. + desc.parent_filters = unsupported.clone(); + } + desc } else { crate::filter_pushdown::ChildFilterDescription { parent_filters: unsupported.clone(), @@ -1131,14 +1150,14 @@ impl ExecutionPlan for HashJoinExec { }; // We only install dynamic filters after optimization to avoid planning scans - // before the hash table provides join key bounds + // before the hash table provides join key bounds. The chosen child was + // analyzed above via `from_child`, meaning it advertises filter pushdown. if matches!(phase, FilterPushdownPhase::Post) && config.optimizer.enable_dynamic_filter_pushdown { if let Some(df) = &self.dynamic_filter { - let target_side = dynamic_filter_side(self.join_type); let dynamic_filter = Arc::clone(df) as Arc; - match target_side { + match dynamic_target { JoinSide::Left => { left_child = left_child.with_self_filter(dynamic_filter); } @@ -1161,6 +1180,10 @@ impl ExecutionPlan for HashJoinExec { child_pushdown_result: ChildPushdownResult, _config: &ConfigOptions, ) -> Result>> { + // `gather_filters_for_pushdown` above marks predicates on non-preserved + // inputs as unsupported. Therefore it is safe to propagate any + // pushdown results returned by the children without re-checking the + // join semantics here. Ok(FilterPushdownPropagation::if_any(child_pushdown_result)) } } @@ -2029,6 +2052,35 @@ mod tests { } } + #[test] + fn preservation_truth_table() { + use JoinType::*; + let cases = [ + Inner, Left, Right, Full, LeftSemi, LeftAnti, RightSemi, RightAnti, LeftMark, + RightMark, + ]; + let table: Vec = cases + .iter() + .map(|jt| { + let lr = lr_is_preserved(*jt); + let on = on_lr_is_preserved(*jt); + format!("{jt:?}: lr={lr:?}, on_lr={on:?}") + }) + .collect(); + assert_snapshot!(table.join("\n"), @r#" +Inner: lr=(true, true), on_lr=(true, true) +Left: lr=(true, false), on_lr=(false, true) +Right: lr=(false, true), on_lr=(true, false) +Full: lr=(false, false), on_lr=(false, false) +LeftSemi: lr=(true, false), on_lr=(true, true) +LeftAnti: lr=(true, false), on_lr=(false, true) +RightSemi: lr=(false, true), on_lr=(true, true) +RightAnti: lr=(false, true), on_lr=(true, false) +LeftMark: lr=(true, false), on_lr=(false, true) +RightMark: lr=(false, true), on_lr=(true, false) +"#); + } + #[template] #[rstest] fn batch_sizes(#[values(8192, 10, 5, 2, 1)] batch_size: usize) {} From ec47ed7eaf00a2e579456b255a644f514a79defa Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 21:47:09 +0800 Subject: [PATCH 32/72] Add join preservation utilities and refactor related functions for clarity --- datafusion/common/src/joins/mod.rs | 20 ++++++ datafusion/common/src/joins/preservation.rs | 67 +++++++++++++++++++ datafusion/common/src/lib.rs | 1 + .../optimizer/src/filter_null_join_keys.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 44 +----------- .../physical-plan/src/joins/hash_join.rs | 34 +--------- 6 files changed, 93 insertions(+), 75 deletions(-) create mode 100644 datafusion/common/src/joins/mod.rs create mode 100644 datafusion/common/src/joins/preservation.rs diff --git a/datafusion/common/src/joins/mod.rs b/datafusion/common/src/joins/mod.rs new file mode 100644 index 000000000000..7a08bd6be03d --- /dev/null +++ b/datafusion/common/src/joins/mod.rs @@ -0,0 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Utilities shared between join implementations. + +pub mod preservation; diff --git a/datafusion/common/src/joins/preservation.rs b/datafusion/common/src/joins/preservation.rs new file mode 100644 index 000000000000..07e54f0e7815 --- /dev/null +++ b/datafusion/common/src/joins/preservation.rs @@ -0,0 +1,67 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Helpers for reasoning about which sides of a [`JoinType`] preserve +//! their input rows. + +use crate::JoinType; + +/// For a given [`JoinType`], determine whether each input of the join is +/// preserved for filters applied *after* the join. +/// +/// A preserved side guarantees that each row in the join output maps back to a +/// row from the preserved input table. If a table is not preserved, it can +/// produce additional rows containing NULL values. For example: +/// +/// * In an [`JoinType::Inner`] join, both sides are preserved because every +/// output row originates from a matching row on each side. +/// * In a [`JoinType::Left`] join, the left side is preserved but the right side +/// is not because the join may output extra rows with NULLs for the right +/// columns when there is no match. +/// +/// The returned tuple is `(left_preserved, right_preserved)`. +pub fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { + match join_type { + JoinType::Inner => (true, true), + JoinType::Left => (true, false), + JoinType::Right => (false, true), + JoinType::Full => (false, false), + // For semi/anti joins the non-driving side cannot appear in the output. + JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => (true, false), + JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => (false, true), + } +} + +/// For a given [`JoinType`], determine whether each input of the join is +/// preserved for filters in the join condition (ON-clause filters). +/// +/// Only preserved sides may safely have filters pushed below the join. +/// +/// The returned tuple is `(left_preserved, right_preserved)`. +pub fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { + match join_type { + JoinType::Inner => (true, true), + JoinType::Left => (false, true), + JoinType::Right => (true, false), + JoinType::Full => (false, false), + JoinType::LeftSemi | JoinType::RightSemi => (true, true), + JoinType::LeftAnti => (false, true), + JoinType::RightAnti => (true, false), + JoinType::LeftMark => (false, true), + JoinType::RightMark => (true, false), + } +} diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 3a558fa86789..4378ae1503b4 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -47,6 +47,7 @@ pub mod file_options; pub mod format; pub mod hash_utils; pub mod instant; +pub mod joins; pub mod nested_struct; mod null_equality; pub mod parsers; diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index 8ad7fa53c0e3..d06c8edcb824 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -18,8 +18,8 @@ //! [`FilterNullJoinKeys`] adds filters to join inputs when input isn't nullable use crate::optimizer::ApplyOrder; -use crate::push_down_filter::on_lr_is_preserved; use crate::{OptimizerConfig, OptimizerRule}; +use datafusion_common::joins::preservation::on_lr_is_preserved; use datafusion_common::tree_node::Transformed; use datafusion_common::{NullEquality, Result}; use datafusion_expr::utils::conjunction; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 2de81671f465..c765161b731a 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -24,6 +24,7 @@ use arrow::datatypes::DataType; use indexmap::IndexSet; use itertools::Itertools; +use datafusion_common::joins::preservation::{lr_is_preserved, on_lr_is_preserved}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -154,49 +155,6 @@ pub struct PushDownFilter {} /// For example: /// - In an inner join, both sides are preserved, because each row of the output /// maps directly to a row from each side. -/// -/// - In a left join, the left side is preserved (we can push predicates) but -/// the right is not, because there may be rows in the output that don't -/// directly map to a row in the right input (due to nulls filling where there -/// is no match on the right). -pub(crate) fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { - match join_type { - JoinType::Inner => (true, true), - JoinType::Left => (true, false), - JoinType::Right => (false, true), - JoinType::Full => (false, false), - // No columns from the right side of the join can be referenced in output - // predicates for semi/anti joins, so whether we specify t/f doesn't matter. - JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => (true, false), - // No columns from the left side of the join can be referenced in output - // predicates for semi/anti joins, so whether we specify t/f doesn't matter. - JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => (false, true), - } -} - -/// For a given JOIN type, determine whether each input of the join is preserved -/// for the join condition (`ON` clause filters). -/// -/// It is only correct to push filters below a join for preserved inputs. -/// -/// # Return Value -/// A tuple of booleans - (left_preserved, right_preserved). -/// -/// See [`lr_is_preserved`] for a definition of "preserved". -pub(crate) fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { - match join_type { - JoinType::Inner => (true, true), - JoinType::Left => (false, true), - JoinType::Right => (true, false), - JoinType::Full => (false, false), - JoinType::LeftSemi | JoinType::RightSemi => (true, true), - JoinType::LeftAnti => (false, true), - JoinType::RightAnti => (true, false), - JoinType::LeftMark => (false, true), - JoinType::RightMark => (true, false), - } -} - /// Evaluates the columns referenced in the given expression to see if they refer /// only to the left or right columns #[derive(Debug)] diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 61290d483443..95e3966d46f6 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -75,8 +75,9 @@ use arrow::util::bit_util; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::memory::estimate_memory_size; use datafusion_common::{ - internal_datafusion_err, internal_err, plan_err, project_schema, JoinSide, JoinType, - NullEquality, Result, ScalarValue, + internal_datafusion_err, internal_err, + joins::preservation::{lr_is_preserved, on_lr_is_preserved}, + plan_err, project_schema, JoinSide, JoinType, NullEquality, Result, ScalarValue, }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; @@ -94,35 +95,6 @@ use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::{ready, Stream, StreamExt, TryStreamExt}; use parking_lot::Mutex; -// Helper functions mirrored from the optimizer's push_down_filter module. -// KEEP IN SYNC WITH datafusion/optimizer/src/push_down_filter.rs -// Determine which sides of a JOIN preserve rows for join output filters -fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { - match join_type { - JoinType::Inner => (true, true), - JoinType::Left => (true, false), - JoinType::Right => (false, true), - JoinType::Full => (false, false), - JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => (true, false), - JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => (false, true), - } -} - -// Determine which sides of a JOIN are preserved for ON-clause filters -fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { - match join_type { - JoinType::Inner => (true, true), - JoinType::Left => (false, true), - JoinType::Right => (true, false), - JoinType::Full => (false, false), - JoinType::LeftSemi | JoinType::RightSemi => (true, true), - JoinType::LeftAnti => (false, true), - JoinType::RightAnti => (true, false), - JoinType::LeftMark => (false, true), - JoinType::RightMark => (true, false), - } -} - /// Returns which side of the join should receive a dynamic filter. /// /// Mark joins apply filters to the *opposite* side of the preserved input so From 5eabb3647efadcd0c43777fb5a704465d77449c4 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 22:04:54 +0800 Subject: [PATCH 33/72] Refactor dynamic filter display logic in HashJoinExec for clarity and consistency. Update condition checks for join types. --- datafusion/common/src/joins/preservation.rs | 50 ++++++++++++++----- .../physical-plan/src/joins/hash_join.rs | 8 +-- 2 files changed, 42 insertions(+), 16 deletions(-) diff --git a/datafusion/common/src/joins/preservation.rs b/datafusion/common/src/joins/preservation.rs index 07e54f0e7815..e4d76fe440e3 100644 --- a/datafusion/common/src/joins/preservation.rs +++ b/datafusion/common/src/joins/preservation.rs @@ -23,17 +23,26 @@ use crate::JoinType; /// For a given [`JoinType`], determine whether each input of the join is /// preserved for filters applied *after* the join. /// -/// A preserved side guarantees that each row in the join output maps back to a -/// row from the preserved input table. If a table is not preserved, it can -/// produce additional rows containing NULL values. For example: +/// Row preservation means every output row can be traced back to a row from +/// that input. Non‑preserved sides may introduce additional NULL padded rows. +/// The table below visualises the behaviour (`✓` preserved, `✗` not preserved): /// -/// * In an [`JoinType::Inner`] join, both sides are preserved because every -/// output row originates from a matching row on each side. -/// * In a [`JoinType::Left`] join, the left side is preserved but the right side -/// is not because the join may output extra rows with NULLs for the right -/// columns when there is no match. +/// ```text +/// left right +/// INNER ✓ ✓ +/// LEFT ✓ ✗ +/// RIGHT ✗ ✓ +/// FULL ✗ ✗ +/// LEFT SEMI ✓ ✗ +/// LEFT ANTI ✓ ✗ +/// LEFT MARK ✓ ✗ +/// RIGHT SEMI ✗ ✓ +/// RIGHT ANTI ✗ ✓ +/// RIGHT MARK ✗ ✓ +/// ``` /// -/// The returned tuple is `(left_preserved, right_preserved)`. +/// The returned tuple `(left_preserved, right_preserved)` reports whether each +/// side of the join preserves its input rows under post‑join filtering. pub fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { match join_type { JoinType::Inner => (true, true), @@ -47,11 +56,28 @@ pub fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { } /// For a given [`JoinType`], determine whether each input of the join is -/// preserved for filters in the join condition (ON-clause filters). +/// preserved for filters in the join condition (ON‑clause filters). /// -/// Only preserved sides may safely have filters pushed below the join. +/// Filters on ON‑clause expressions may only reference sides that are +/// preserved; otherwise pushing the filter below the join could drop rows. +/// This table shows preservation for ON‑clause evaluation (`✓` preserved): /// -/// The returned tuple is `(left_preserved, right_preserved)`. +/// ```text +/// left right +/// INNER ✓ ✓ +/// LEFT ✗ ✓ +/// RIGHT ✓ ✗ +/// FULL ✗ ✗ +/// LEFT SEMI ✓ ✓ +/// RIGHT SEMI ✓ ✓ +/// LEFT ANTI ✗ ✓ +/// RIGHT ANTI ✓ ✗ +/// LEFT MARK ✗ ✓ +/// RIGHT MARK ✓ ✗ +/// ``` +/// +/// The returned tuple `(left_preserved, right_preserved)` reports which sides +/// may safely participate in ON‑clause filtering. pub fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { match join_type { JoinType::Inner => (true, true), diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 95e3966d46f6..abc396bc7794 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -719,7 +719,7 @@ impl DisplayAs for HashJoinExec { .map(|(c1, c2)| format!("({c1}, {c2})")) .collect::>() .join(", "); - let dynamic_filter_display = match &self.dynamic_filter { + let probe_filter_display = match &self.dynamic_filter { Some(df) => { let probe_side = dynamic_filter_side(self.join_type); let keys = df.key_count(); @@ -746,7 +746,7 @@ impl DisplayAs for HashJoinExec { on, display_filter, display_projections, - dynamic_filter_display + probe_filter_display ) } DisplayFormatType::TreeRender => { @@ -759,7 +759,7 @@ impl DisplayAs for HashJoinExec { .collect::>() .join(", "); - if *self.join_type() != JoinType::Inner { + if !matches!(self.join_type(), JoinType::Inner) { writeln!(f, "join_type={:?}", self.join_type)?; } writeln!(f, "on={on}") @@ -1866,7 +1866,7 @@ impl HashJoinStream { self.right_side_ordered, )?; - let result = if self.join_type == JoinType::RightMark { + let result = if matches!(self.join_type, JoinType::RightMark) { build_batch_from_indices( &self.schema, &state.batch, From 014bed8707457ac59a25002658ef6d46bb8bfc86 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 22:45:21 +0800 Subject: [PATCH 34/72] Refactor dynamic filter handling in joins and update related documentation. Rename preservation functions for clarity and enhance configuration descriptions for dynamic filter pushdown. --- datafusion/common/src/config.rs | 30 ++++++++------- datafusion/common/src/joins/mod.rs | 4 ++ datafusion/common/src/joins/preservation.rs | 12 +++++- .../optimizer/src/filter_null_join_keys.rs | 4 +- datafusion/optimizer/src/push_down_filter.rs | 12 ++++-- .../physical-plan/src/joins/hash_join.rs | 17 ++++++--- docs/source/library-user-guide/upgrading.md | 37 +++++++++++-------- docs/source/user-guide/configs.md | 2 +- 8 files changed, 75 insertions(+), 43 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 8c98786886ef..2c44f65e87d1 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -733,28 +733,32 @@ config_namespace! { /// year 2025 any files that only have timestamps in the year 2024 can be skipped / /// pruned at various stages in the scan. /// - /// Dynamic filters are also produced by left, right, semi, and anti joins, - /// allowing DataFusion to prune the probe side during execution. The *probe - /// side* is the stream that is scanned in a hash join. For hash joins the - /// mapping is: + /// Dynamic filters are also produced by joins, allowing DataFusion to prune + /// the *probe* side during execution. `HashJoinExec` builds from its left + /// input and probes with its right input. Legend: Left = left child, Right = + /// right child. /// /// | Join type | Probe side | - /// |-------------------------|-----------| + /// |-------------------------|------------| /// | `Inner`, `Left` | Right input | - /// | `Right` | Left input | - /// | `LeftSemi`, `LeftAnti` | Left input | + /// | `Right` | Left input | + /// | `LeftSemi`, `LeftAnti` | Left input | /// | `RightSemi`, `RightAnti`| Right input | /// /// Full joins are not supported. /// - /// Non-equi join predicates do **not** generate dynamic filters as they require - /// range or conjunctive analysis; composite predicates only derive filters from - /// their equi-conjuncts. + /// Non-equi join predicates do **not** generate dynamic filters; they require + /// range analysis and cross-conjunct reasoning (future work). Composite + /// predicates only derive filters from their equi-conjuncts, and rows with + /// `NULL` join keys (see [`crate::NullEquality::NullEqualsNothing`]) do not contribute + /// filter values. Enabling `optimizer.filter_null_join_keys` can remove such + /// rows up front. /// /// Pushdown is effective only when the file source supports predicate pushdown - /// (e.g. Parquet) and `execution.parquet.pushdown_filters` is enabled. - /// See the upgrade guide for additional details and examples. - /// For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` + /// (e.g. Parquet) and `execution.parquet.pushdown_filters` is `true`; formats + /// without predicate pushdown (CSV/JSON) see no benefit. See the upgrade guide + /// for additional details and examples. For example, + /// `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` /// will only read `fact` rows whose `id` values match `dim` rows where /// `region = 'US'`. pub enable_dynamic_filter_pushdown: bool, default = true diff --git a/datafusion/common/src/joins/mod.rs b/datafusion/common/src/joins/mod.rs index 7a08bd6be03d..4982fb92f4cb 100644 --- a/datafusion/common/src/joins/mod.rs +++ b/datafusion/common/src/joins/mod.rs @@ -18,3 +18,7 @@ //! Utilities shared between join implementations. pub mod preservation; + +pub use preservation::{ + preservation_for_on_filters, preservation_for_output_filters, +}; diff --git a/datafusion/common/src/joins/preservation.rs b/datafusion/common/src/joins/preservation.rs index e4d76fe440e3..3d93f047e9be 100644 --- a/datafusion/common/src/joins/preservation.rs +++ b/datafusion/common/src/joins/preservation.rs @@ -17,6 +17,7 @@ //! Helpers for reasoning about which sides of a [`JoinType`] preserve //! their input rows. +//! **`preservation_for_output_filters` answers post‑join output filtering, whereas `preservation_for_on_filters` addresses ON‑clause feasibility.** use crate::JoinType; @@ -43,7 +44,8 @@ use crate::JoinType; /// /// The returned tuple `(left_preserved, right_preserved)` reports whether each /// side of the join preserves its input rows under post‑join filtering. -pub fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { +#[inline] +pub const fn preservation_for_output_filters(join_type: JoinType) -> (bool, bool) { match join_type { JoinType::Inner => (true, true), JoinType::Left => (true, false), @@ -78,7 +80,8 @@ pub fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { /// /// The returned tuple `(left_preserved, right_preserved)` reports which sides /// may safely participate in ON‑clause filtering. -pub fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { +#[inline] +pub const fn preservation_for_on_filters(join_type: JoinType) -> (bool, bool) { match join_type { JoinType::Inner => (true, true), JoinType::Left => (false, true), @@ -91,3 +94,8 @@ pub fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { JoinType::RightMark => (true, false), } } + +#[allow(unused_imports)] +pub(crate) use preservation_for_output_filters as lr_is_preserved; +#[allow(unused_imports)] +pub(crate) use preservation_for_on_filters as on_lr_is_preserved; diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index d06c8edcb824..a14635f872e4 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -19,7 +19,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::joins::preservation::on_lr_is_preserved; +use datafusion_common::joins::preservation_for_on_filters; use datafusion_common::tree_node::Transformed; use datafusion_common::{NullEquality, Result}; use datafusion_expr::utils::conjunction; @@ -55,7 +55,7 @@ impl OptimizerRule for FilterNullJoinKeys { && join.null_equality == NullEquality::NullEqualsNothing => { let (left_preserved, right_preserved) = - on_lr_is_preserved(join.join_type); + preservation_for_on_filters(join.join_type); let left_schema = join.left.schema(); let right_schema = join.right.schema(); diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index c765161b731a..258c61231840 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -24,7 +24,11 @@ use arrow::datatypes::DataType; use indexmap::IndexSet; use itertools::Itertools; -use datafusion_common::joins::preservation::{lr_is_preserved, on_lr_is_preserved}; +// `preservation_for_output_filters` determines output-filter safety; `preservation_for_on_filters` +// governs ON-clause pushdown. +use datafusion_common::joins::{ + preservation_for_on_filters, preservation_for_output_filters, +}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -384,7 +388,8 @@ fn push_down_all_join( ) -> Result> { let is_inner_join = join.join_type == JoinType::Inner; // Get pushable predicates from current optimizer state - let (left_preserved, right_preserved) = lr_is_preserved(join.join_type); + let (left_preserved, right_preserved) = + preservation_for_output_filters(join.join_type); // The predicates can be divided to three categories: // 1) can push through join to its children(left or right) @@ -421,7 +426,8 @@ fn push_down_all_join( } let mut on_filter_join_conditions = vec![]; - let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join.join_type); + let (on_left_preserved, on_right_preserved) = + preservation_for_on_filters(join.join_type); if !on_filter.is_empty() { for on in on_filter { diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index abc396bc7794..229101abb09d 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -76,7 +76,9 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::utils::memory::estimate_memory_size; use datafusion_common::{ internal_datafusion_err, internal_err, - joins::preservation::{lr_is_preserved, on_lr_is_preserved}, + joins::{ + preservation_for_on_filters, preservation_for_output_filters, + }, plan_err, project_schema, JoinSide, JoinType, NullEquality, Result, ScalarValue, }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -100,8 +102,10 @@ use parking_lot::Mutex; /// Mark joins apply filters to the *opposite* side of the preserved input so /// that only rows capable of satisfying the ON clause are evaluated. fn dynamic_filter_side(join_type: JoinType) -> JoinSide { - let (left_preserved, right_preserved) = lr_is_preserved(join_type); - let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join_type); + let (left_preserved, right_preserved) = + preservation_for_output_filters(join_type); + let (on_left_preserved, on_right_preserved) = + preservation_for_on_filters(join_type); match ( left_preserved, right_preserved, @@ -1070,7 +1074,8 @@ impl ExecutionPlan for HashJoinExec { parent_filters: Vec>, config: &ConfigOptions, ) -> Result { - let (left_preserved, right_preserved) = lr_is_preserved(self.join_type); + let (left_preserved, right_preserved) = + preservation_for_output_filters(self.join_type); let dynamic_target = dynamic_filter_side(self.join_type); let unsupported: Vec<_> = parent_filters @@ -2034,8 +2039,8 @@ mod tests { let table: Vec = cases .iter() .map(|jt| { - let lr = lr_is_preserved(*jt); - let on = on_lr_is_preserved(*jt); + let lr = preservation_for_output_filters(*jt); + let on = preservation_for_on_filters(*jt); format!("{jt:?}: lr={lr:?}, on_lr={on:?}") }) .collect(); diff --git a/docs/source/library-user-guide/upgrading.md b/docs/source/library-user-guide/upgrading.md index 3ace9dbf502d..e54b4771ab03 100644 --- a/docs/source/library-user-guide/upgrading.md +++ b/docs/source/library-user-guide/upgrading.md @@ -42,25 +42,30 @@ returns a `ColumnarValue` instead of a `ArrayRef`. ### Dynamic filter pushdown for joins -Dynamic filter pushdown now applies to left, right, semi and anti joins, +Dynamic filter pushdown now applies to `LEFT`, `RIGHT`, `SEMI` and `ANTI` joins, allowing DataFusion to prune the probe side as join keys are discovered at -runtime. Full joins are not supported and only equi-join keys contribute to the -filters. Dynamic filters obey `NullEqualsNothing` semantics, so rows with null -join keys never match and generate no filter values. This behavior is controlled -by the `datafusion.optimizer.enable_dynamic_filter_pushdown` configuration -option (on by default). Dynamic filter pushdown requires file formats that -support predicate pushdown; for Parquet this means enabling -`datafusion.execution.parquet.pushdown_filters`. - -In a hash join, the probe side is the input whose rows are scanned to find -matches against the hashed build side. +runtime. `HashJoinExec` builds from its left child and probes with its right +child (Left = left child, Right = right child). Full joins are not supported and +only equi-join keys contribute. Non-equi predicates require range analysis and +cross-conjunct reasoning (future work). Dynamic filters obey +`NullEqualsNothing` semantics, so rows with null join keys never match and +generate no filter values. Consider enabling +`datafusion.optimizer.filter_null_join_keys` to remove nulls early. This +behavior is controlled by the +`datafusion.optimizer.enable_dynamic_filter_pushdown` configuration option (on +by default). + +Dynamic filter pushdown requires file formats that support predicate pushdown; +for Parquet this means enabling +`datafusion.execution.parquet.pushdown_filters`. Formats such as CSV or JSON do +not benefit. | JoinType | Probe side pruned | -| ------------------------ | ----------------- | -| `Inner`, `Left` | Right input | -| `Right` | Left input | -| `LeftSemi`, `LeftAnti` | Left input | -| `RightSemi`, `RightAnti` | Right input | +| ------------------------ | ---------------- | +| `Inner`, `Left` | Right input | +| `Right` | Left input | +| `LeftSemi`, `LeftAnti` | Left input | +| `RightSemi`, `RightAnti` | Right input | Dynamic filters are most effective when the join keys are highly selective. You can disable the feature by setting diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 11b578ac2d08..6ca6091bbf89 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -129,7 +129,7 @@ The following configuration settings are available: | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by left, right, semi, and anti joins into the file scan phase, pruning the probe side. Full joins are not supported and only equi-join keys generate filters. See the [join-type table](../library-user-guide/upgrading.md#dynamic-filter-pushdown-for-joins) for details. Disable for high-cardinality keys or file formats without predicate pushdown (e.g., CSV/JSON). Interacts with `execution.parquet.pushdown_filters`. | +| `datafusion.optimizer.enable_dynamic_filter_pushdown` | true | Pushes down dynamic filters produced by joins to prune the probe side (`HashJoinExec` builds left, probes right). Only equi-join keys contribute; see the [join-type table](../library-user-guide/upgrading.md#dynamic-filter-pushdown-for-joins). Formats without predicate pushdown (CSV/JSON) see no benefit. Interacts with `execution.parquet.pushdown_filters` and `optimizer.filter_null_join_keys`. | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | | datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | From 92ed50df773d4de06dad27c6558865e610c98aa0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 23:10:10 +0800 Subject: [PATCH 35/72] Enhance dynamic filter observability and error handling. Add inline comments for clarity, use relaxed atomics for key count updates, and refactor filter information display in SortExec. --- .../src/expressions/dynamic_filters.rs | 38 +++++++++---- .../physical-optimizer/src/filter_pushdown.rs | 5 ++ .../physical-plan/src/joins/hash_join.rs | 22 +++++--- datafusion/physical-plan/src/sorts/sort.rs | 53 ++++++++----------- datafusion/physical-plan/src/topk/mod.rs | 3 +- 5 files changed, 73 insertions(+), 48 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index 00a5540b3010..548385037515 100644 --- a/datafusion/physical-expr/src/expressions/dynamic_filters.rs +++ b/datafusion/physical-expr/src/expressions/dynamic_filters.rs @@ -52,6 +52,7 @@ pub struct DynamicFilterPhysicalExpr { /// The source of dynamic filters. inner: Arc>, /// Number of keys currently contained in this dynamic filter. + /// Uses relaxed atomics as this counter is for diagnostics only. key_count: Arc, /// For testing purposes track the data type and nullability to make sure they don't change. /// If they do, there's a bug in the implementation. @@ -149,6 +150,7 @@ impl DynamicFilterPhysicalExpr { } } + #[inline] fn remap_children( children: &[Arc], remapped_children: Option<&Vec>>, @@ -181,14 +183,15 @@ impl DynamicFilterPhysicalExpr { /// Get the current expression. /// This will return the current expression with any children /// remapped to match calls to [`PhysicalExpr::with_new_children`]. + #[inline] pub fn current(&self) -> Result> { let inner = Arc::clone( &self .inner .read() - .map_err(|_| { + .map_err(|e| { datafusion_common::DataFusionError::Execution( - "Failed to acquire read lock for inner".to_string(), + format!("Failed to acquire read lock for inner: {e}"), ) })? .expr, @@ -204,16 +207,22 @@ impl DynamicFilterPhysicalExpr { /// This should be called e.g.: /// - When we've computed the probe side's hash table in a HashJoinExec /// - After every batch is processed if we update the TopK heap in a SortExec using a TopK approach. + /// `key_count` specifies the number of keys currently tracked by the filter + /// and is used for observability. + #[inline] pub fn update( &self, new_expr: Arc, key_count: usize, ) -> Result<()> { - let mut current = self.inner.write().map_err(|_| { - datafusion_common::DataFusionError::Execution( - "Failed to acquire write lock for inner".to_string(), - ) - })?; + let mut current = self + .inner + .write() + .map_err(|e| { + datafusion_common::DataFusionError::Execution( + format!("Failed to acquire write lock for inner: {e}"), + ) + })?; // Remap the children of the new expression to match the original children // We still do this again in `current()` but doing it preventively here // reduces the work needed in some cases if `current()` is called multiple times @@ -227,13 +236,24 @@ impl DynamicFilterPhysicalExpr { current.expr = new_expr; // Increment the generation to indicate that the expression has changed. current.generation += 1; - self.key_count.store(key_count, Ordering::SeqCst); + // Relaxed ordering is sufficient as `key_count` is only used for + // observability and does not synchronize with other data. + self.key_count.store(key_count, Ordering::Relaxed); Ok(()) } + /// Update the inner expression without changing the key count. + #[deprecated(note = "use `update` with an explicit key count instead")] + #[inline] + pub fn update_expr(&self, expr: Arc) -> Result<()> { + self.update(expr, 0) + } + /// Return the current number of keys represented by this dynamic filter. + #[inline] pub fn key_count(&self) -> usize { - self.key_count.load(Ordering::SeqCst) + // See note in `update`; relaxed ordering is sufficient. + self.key_count.load(Ordering::Relaxed) } } diff --git a/datafusion/physical-optimizer/src/filter_pushdown.rs b/datafusion/physical-optimizer/src/filter_pushdown.rs index 2838d1d1f243..1f515f03f9bf 100644 --- a/datafusion/physical-optimizer/src/filter_pushdown.rs +++ b/datafusion/physical-optimizer/src/filter_pushdown.rs @@ -501,6 +501,9 @@ fn push_down_filters( let mut all_predicates = self_filtered.items().to_vec(); // Apply second filter pass: collect indices of parent filters that can be pushed down + // Invariant: predicates marked as unsupported by `gather_filters_for_pushdown` + // must remain so downstream. We only forward predicates explicitly marked + // as `PushedDown::Yes` here and never re-enable previously rejected ones. let parent_filters_for_child = parent_filtered .chain_filter_slice(&parent_filters, |filter| { matches!(filter.discriminant, PushedDown::Yes) @@ -708,6 +711,7 @@ impl FilteredVec { } } +#[inline] fn allow_pushdown_for_expr(expr: &Arc) -> bool { let mut allow_pushdown = true; expr.apply(|e| { @@ -722,6 +726,7 @@ fn allow_pushdown_for_expr(expr: &Arc) -> bool { allow_pushdown } +#[inline] fn allow_pushdown_for_expr_inner(expr: &Arc) -> bool { if let Some(scalar_function) = expr.as_any() diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 229101abb09d..e968856abf50 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -99,8 +99,10 @@ use parking_lot::Mutex; /// Returns which side of the join should receive a dynamic filter. /// +// Inner joins choose the right (probe) side for determinism. /// Mark joins apply filters to the *opposite* side of the preserved input so /// that only rows capable of satisfying the ON clause are evaluated. +#[inline] fn dynamic_filter_side(join_type: JoinType) -> JoinSide { let (left_preserved, right_preserved) = preservation_for_output_filters(join_type); @@ -187,6 +189,7 @@ impl JoinLeftData { /// Decrements the counter of running threads, and returns `true` /// if caller is the last running thread + #[inline] fn report_probe_completed(&self) -> bool { self.probe_threads_counter.fetch_sub(1, Ordering::Relaxed) == 1 } @@ -414,10 +417,11 @@ impl HashJoinExec { ) -> Result { let left_schema = left.schema(); let right_schema = right.schema(); - assert!( - !on.is_empty(), - "HashJoinExec requires a non-empty ON clause; empty lists are unsupported" - ); + if on.is_empty() { + return plan_err!( + "HashJoinExec requires a non-empty ON clause; empty lists are unsupported" + ); + } check_join_is_valid(&left_schema, &right_schema, &on)?; @@ -462,6 +466,7 @@ impl HashJoinExec { }) } + #[inline] fn create_dynamic_filter( on: &JoinOn, join_type: JoinType, @@ -847,10 +852,11 @@ impl ExecutionPlan for HashJoinExec { self.mode, self.null_equality, )?; - // Preserve the dynamic filter if it exists. - // The `on` clause is unchanged so the filter keys remain valid; if the - // `on` clause changes, `try_new` recomputes the keys via - // `create_dynamic_filter` to avoid stale filters. + // Preserve the dynamic filter if it exists. Cloning the `Option>` + // is safe because the `on` clause is unchanged and thus the filter keys + // remain valid. If the `on` clause were to change, `try_new` would + // recompute a fresh filter via `create_dynamic_filter` to avoid + // carrying stale keys. new_join.dynamic_filter = self.dynamic_filter.clone(); Ok(Arc::new(new_join)) } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index dd798585f25c..8c4d4b1d0dbf 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -78,6 +78,27 @@ impl ExternalSorterMetrics { } } +#[inline] +fn write_filter_info( + f: &mut Formatter, + filter: &Option>, +) -> fmt::Result { + if let Some(filter) = filter { + let keys = filter.key_count(); + if let Ok(current) = filter.current() { + if !current.eq(&lit(true)) { + write!(f, ", filter=[{current}]")?; + write!(f, ", filter_keys={keys}")?; + return Ok(()); + } + } + if keys > 0 { + write!(f, ", filter_keys={keys}")?; + } + } + Ok(()) +} + /// Sorts an arbitrary sized, unsorted, stream of [`RecordBatch`]es to /// a total order. Depending on the input size and memory manager /// configuration, writes intermediate results to disk ("spills") @@ -1050,21 +1071,7 @@ impl DisplayAs for SortExec { match self.fetch { Some(fetch) => { write!(f, "SortExec: TopK(fetch={fetch}), expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr)?; - if let Some(filter) = &self.filter { - let keys = filter.key_count(); - if let Ok(current) = filter.current() { - if !current.eq(&lit(true)) { - write!( - f, - ", filter=[{current}], filter_keys={keys}" - )?; - } else { - write!(f, ", filter_keys={keys}")?; - } - } else { - write!(f, ", filter_keys={keys}")?; - } - } + write_filter_info(f, &self.filter)?; if !self.common_sort_prefix.is_empty() { write!(f, ", sort_prefix=[")?; let mut first = true; @@ -1083,21 +1090,7 @@ impl DisplayAs for SortExec { } None => { write!(f, "SortExec: expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr)?; - if let Some(filter) = &self.filter { - let keys = filter.key_count(); - if let Ok(current) = filter.current() { - if !current.eq(&lit(true)) { - write!( - f, - ", filter=[{current}], filter_keys={keys}" - )?; - } else { - write!(f, ", filter_keys={keys}")?; - } - } else { - write!(f, ", filter_keys={keys}")?; - } - } + write_filter_info(f, &self.filter)?; Ok(()) } } diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index d3bdac2384a7..0c5ef4453ecb 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -406,7 +406,8 @@ impl TopK { if let Some(predicate) = dynamic_predicate { filter.update(predicate, self.heap.len())?; } else { - // even if predicate is true, record current key count + // Even when the dynamic predicate is a tautology we still update + // `key_count` for observability. filter.update(lit(true), self.heap.len())?; } From 853fa6c92f4d18d38c462c1cf71b0b848b1b38d6 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Wed, 13 Aug 2025 23:55:49 +0800 Subject: [PATCH 36/72] Enhance dynamic filter pushdown tests and output formatting. Update assertions to check for probe keys and modify predicate representation to use `` for clarity in test outputs. --- .../physical_optimizer/filter_pushdown/mod.rs | 62 ++++++++++++++++--- .../filter_pushdown/util.rs | 8 ++- .../physical-plan/src/joins/hash_join.rs | 53 ++++++++++------ .../sqllogictest/test_files/explain_tree.slt | 2 +- 4 files changed, 94 insertions(+), 31 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 221e3a1ceff6..e5c019f59f5e 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -274,6 +274,9 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= aa AND d@0 <= ab ] AND DynamicFilterPhysicalExpr [ e@1 IS NULL OR e@1 < bb ] " ); + let formatted = format_plan_for_test(&plan); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } // Test both static and dynamic filter pushdown in HashJoinExec. @@ -848,6 +851,10 @@ async fn test_hashjoin_dynamic_filter_pushdown() { " ); + let formatted = format_plan_for_test(&plan); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); + let join = plan.as_any().downcast_ref::().unwrap(); assert_projection(&join.left, &["a", "b", "c"]); assert_projection(&join.right, &["a", "b", "e"]); @@ -934,10 +941,15 @@ async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { stream.next().await.unwrap().unwrap(); let formatted = format_plan_for_test(&plan); assert_contains!( - formatted, + &formatted, "DynamicFilterPhysicalExpr [ a@0 >= NULL AND a@0 <= NULL AND b@1 >= NULL AND b@1 <= NULL ]" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); + + // TODO: NullEqualsNothing yields no matching keys, so the dynamic filter + // becomes a tautology. let join = plan.as_any().downcast_ref::().unwrap(); assert_projection(&join.left, &["a", "b", "c"]); assert_projection(&join.right, &["a", "b", "e"]); @@ -949,6 +961,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { use arrow::record_batch::RecordBatch; use datafusion_common::JoinType; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; + use tokio::time::{timeout, Duration}; // Generate large key sets to watch for planning regressions let size = 10_000; @@ -1014,11 +1027,17 @@ async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); + let plan = timeout(Duration::from_secs(10), async { + FilterPushdown::new_post_optimization().optimize(plan, &config) + }) + .await + .expect("optimizer should finish in time") + .unwrap(); let formatted = format_plan_for_test(&plan); - assert_contains!(formatted, "DynamicFilterPhysicalExpr"); + assert_contains!(&formatted, "DynamicFilterPhysicalExpr"); + + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); let join = plan.as_any().downcast_ref::().unwrap(); assert_projection(&join.left, &["a", "x"]); @@ -1172,6 +1191,10 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { " ); + let formatted = format_plan_for_test(&plan); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(formatted, "probe_keys=0"); + let outer = plan.as_any().downcast_ref::().unwrap(); assert_projection(&outer.left, &["a", "x"]); let inner = outer.right.as_any().downcast_ref::().unwrap(); @@ -1385,7 +1408,8 @@ async fn test_hashjoin_non_equi_predicate_no_dynamic_filter() { PartitionMode::Partitioned, datafusion_common::NullEquality::NullEqualsNothing, ); - assert!(res.is_err(), "non-equi joins should be rejected"); + let err = res.expect_err("non-equi joins should be rejected"); + assert_contains!(err.to_string(), "non-equi"); } #[tokio::test] @@ -1411,6 +1435,8 @@ async fn test_hashjoin_left_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1454,6 +1480,8 @@ async fn test_hashjoin_right_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1497,6 +1525,8 @@ async fn test_hashjoin_left_dynamic_filter_pushdown_collect_left() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1522,6 +1552,8 @@ async fn test_hashjoin_left_semi_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1547,6 +1579,8 @@ async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1572,6 +1606,8 @@ async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1597,6 +1633,8 @@ async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1616,12 +1654,14 @@ async fn test_hashjoin_left_mark_dynamic_filter_pushdown() { let formatted = format_plan_for_test(&plan); assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr", + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=", + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1641,12 +1681,14 @@ async fn test_hashjoin_right_mark_dynamic_filter_pushdown() { let formatted = format_plan_for_test(&plan); assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr", + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=", + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); } #[tokio::test] diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 74239f5f0b23..885b32c092c1 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -195,6 +195,8 @@ impl FileSource for TestSource { DisplayFormatType::Default | DisplayFormatType::Verbose => { let support = format!(", pushdown_supported={}", self.support); + // Use `` to denote the absence of a predicate so that + // test output remains stable across formats. let predicate_string = self .predicate .as_ref() @@ -204,9 +206,13 @@ impl FileSource for TestSource { write!(f, "{support}{predicate_string}") } DisplayFormatType::TreeRender => { + // Tree format also uses `` as the placeholder for + // missing predicates. + writeln!(f, "pushdown_supported={}", self.support)?; if let Some(predicate) = &self.predicate { - writeln!(f, "pushdown_supported={}", fmt_sql(predicate.as_ref()))?; writeln!(f, "predicate={}", fmt_sql(predicate.as_ref()))?; + } else { + writeln!(f, "predicate=")?; } Ok(()) } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index e968856abf50..061b472fff4a 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -771,7 +771,18 @@ impl DisplayAs for HashJoinExec { if !matches!(self.join_type(), JoinType::Inner) { writeln!(f, "join_type={:?}", self.join_type)?; } - writeln!(f, "on={on}") + writeln!(f, "on={on}")?; + if let Some(df) = &self.dynamic_filter { + if let Ok(current) = df.current() { + if current != lit(true) { + writeln!(f, "probe_filter={}", fmt_sql(current.as_ref()))?; + } + } + let probe_side = dynamic_filter_side(self.join_type); + writeln!(f, "probe_side={probe_side:?}")?; + writeln!(f, "probe_keys={}", df.key_count())?; + } + Ok(()) } } } @@ -1074,6 +1085,7 @@ impl ExecutionPlan for HashJoinExec { } } + #[inline] fn gather_filters_for_pushdown( &self, phase: FilterPushdownPhase, @@ -1084,23 +1096,29 @@ impl ExecutionPlan for HashJoinExec { preservation_for_output_filters(self.join_type); let dynamic_target = dynamic_filter_side(self.join_type); - let unsupported: Vec<_> = parent_filters - .iter() - .map(|f| { - crate::filter_pushdown::PushedDownPredicate::unsupported(Arc::clone(f)) - }) - .collect(); + // Prepare a single vector of unsupported predicates to avoid + // rebuilding it for each child. It will be cloned only when both + // sides require it. + let unsupported = if !left_preserved || !right_preserved { + parent_filters + .iter() + .map(|f| { + crate::filter_pushdown::PushedDownPredicate::unsupported(Arc::clone(f)) + }) + .collect::>() + } else { + vec![] + }; - let mut left_child = if left_preserved || matches!(dynamic_target, JoinSide::Left) - { + let mut left_child = if left_preserved || matches!(dynamic_target, JoinSide::Left) { let mut desc = crate::filter_pushdown::ChildFilterDescription::from_child( &parent_filters, self.left(), )?; if !left_preserved { - // For joins like `RightAnti`, the left input is not preserved and - // parent filters can't be enforced. Mark them as unsupported so - // they will not be propagated further down. + // For semi/anti joins the left (non-driving) side cannot + // contribute columns to the output filters, so parent filters + // referencing it are marked unsupported. desc.parent_filters = unsupported.clone(); } desc @@ -1111,23 +1129,20 @@ impl ExecutionPlan for HashJoinExec { } }; - let mut right_child = if right_preserved - || matches!(dynamic_target, JoinSide::Right) - { + let mut right_child = if right_preserved || matches!(dynamic_target, JoinSide::Right) { let mut desc = crate::filter_pushdown::ChildFilterDescription::from_child( &parent_filters, self.right(), )?; if !right_preserved { - // A `LeftAnti` join discards all rows from the right side, so - // any parent filter referencing it would be meaningless. Mark - // such filters unsupported to avoid incorrect pushdown. + // Semi/anti joins discard the right side; parent filters on + // this non-driving side are therefore marked unsupported. desc.parent_filters = unsupported.clone(); } desc } else { crate::filter_pushdown::ChildFilterDescription { - parent_filters: unsupported.clone(), + parent_filters: unsupported, self_filters: vec![], } }; diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 550d06603b1a..7f531ebd59dc 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1241,7 +1241,7 @@ physical_plan 31)│ -------------------- ││ -------------------- │ 32)│ files: 1 ││ partition_count(in->out): │ 33)│ format: parquet ││ 1 -> 4 │ -34)│ predicate: true ││ │ +34)│ predicate: ││ │ 35)│ ││ partitioning_scheme: │ 36)│ ││ RoundRobinBatch(4) │ 37)└───────────────────────────┘└─────────────┬─────────────┘ From 5d2b60c40c30f59a9c775dd8fd83c646183a2e0d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 00:12:05 +0800 Subject: [PATCH 37/72] Refactor dynamic filter handling and enhance documentation. Clean up code formatting in `hash_join.rs`, add join preservation details to user guide, and update configuration documentation for dynamic filter pushdown. --- .../physical-plan/src/joins/hash_join.rs | 26 ++++----- dev/changelog/50.0.0.md | 17 +++--- docs/source/library-user-guide/index.md | 5 ++ .../library-user-guide/join-preservation.md | 54 +++++++++++++++++++ docs/source/library-user-guide/upgrading.md | 25 ++++++--- docs/source/user-guide/configs.md | 4 +- 6 files changed, 103 insertions(+), 28 deletions(-) create mode 100644 docs/source/library-user-guide/join-preservation.md diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 061b472fff4a..8d2d93668e06 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -76,9 +76,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::utils::memory::estimate_memory_size; use datafusion_common::{ internal_datafusion_err, internal_err, - joins::{ - preservation_for_on_filters, preservation_for_output_filters, - }, + joins::{preservation_for_on_filters, preservation_for_output_filters}, plan_err, project_schema, JoinSide, JoinType, NullEquality, Result, ScalarValue, }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -98,16 +96,13 @@ use futures::{ready, Stream, StreamExt, TryStreamExt}; use parking_lot::Mutex; /// Returns which side of the join should receive a dynamic filter. -/// -// Inner joins choose the right (probe) side for determinism. +/// `Inner` joins choose the right (probe) side for determinism. /// Mark joins apply filters to the *opposite* side of the preserved input so -/// that only rows capable of satisfying the ON clause are evaluated. +/// that only rows capable of satisfying the `ON` clause are evaluated. #[inline] fn dynamic_filter_side(join_type: JoinType) -> JoinSide { - let (left_preserved, right_preserved) = - preservation_for_output_filters(join_type); - let (on_left_preserved, on_right_preserved) = - preservation_for_on_filters(join_type); + let (left_preserved, right_preserved) = preservation_for_output_filters(join_type); + let (on_left_preserved, on_right_preserved) = preservation_for_on_filters(join_type); match ( left_preserved, right_preserved, @@ -1103,14 +1098,17 @@ impl ExecutionPlan for HashJoinExec { parent_filters .iter() .map(|f| { - crate::filter_pushdown::PushedDownPredicate::unsupported(Arc::clone(f)) + crate::filter_pushdown::PushedDownPredicate::unsupported(Arc::clone( + f, + )) }) .collect::>() } else { vec![] }; - let mut left_child = if left_preserved || matches!(dynamic_target, JoinSide::Left) { + let mut left_child = if left_preserved || matches!(dynamic_target, JoinSide::Left) + { let mut desc = crate::filter_pushdown::ChildFilterDescription::from_child( &parent_filters, self.left(), @@ -1129,7 +1127,9 @@ impl ExecutionPlan for HashJoinExec { } }; - let mut right_child = if right_preserved || matches!(dynamic_target, JoinSide::Right) { + let mut right_child = if right_preserved + || matches!(dynamic_target, JoinSide::Right) + { let mut desc = crate::filter_pushdown::ChildFilterDescription::from_child( &parent_filters, self.right(), diff --git a/dev/changelog/50.0.0.md b/dev/changelog/50.0.0.md index c1e8926614fa..011ee66c406b 100644 --- a/dev/changelog/50.0.0.md +++ b/dev/changelog/50.0.0.md @@ -21,14 +21,19 @@ under the License. This release is under active development. -- Enable dynamic filter pushdown for left, right, semi, anti, and mark joins [#16445](https://github.com/apache/datafusion/pull/16445) (adriangb). - This non-breaking change is enabled by default and supports pruning the probe side of `Inner`, `Left`, `Right`, `LeftSemi`, `LeftAnti`, `RightSemi`, `RightAnti`, `LeftMark`, and `RightMark` hash joins. - For example, in star-schema workloads joining `dim` tables to a large `fact` table, dynamic filters can drop entire partitions of the `fact` table before they are scanned. - Full joins and non‑equi (range or composite) predicates are not yet supported; see [#7955](https://github.com/apache/datafusion/issues/7955). - Dynamic filters add planning overhead for high-cardinality keys; disable via the environment variable: +- **Breaking:** `DynamicFilterPhysicalExpr::update` now requires an extra + `key_count` argument. +- Enable dynamic filter pushdown for left, right, semi, anti, and mark joins + [#16445](https://github.com/apache/datafusion/pull/16445) (adriangb). Mark joins + push filters to the side opposite the preserved input (`dynamic_filter_side`; see + [tests](https://github.com/apache/datafusion/blob/main/datafusion/physical-plan/src/joins/hash_join.rs#L2033-L2049)). Formats without predicate pushdown (CSV/JSON) will not benefit. + Full joins and non‑equi (range or composite) predicates are not yet supported; + see [#7955](https://github.com/apache/datafusion/issues/7955). Dynamic filters + add planning overhead for high-cardinality keys; disable via: ```bash export DATAFUSION_OPTIMIZER_ENABLE_DYNAMIC_FILTER_PUSHDOWN=0 ``` - or by setting `datafusion.optimizer.enable_dynamic_filter_pushdown=false` programmatically if necessary. + or by setting `datafusion.optimizer.enable_dynamic_filter_pushdown=false` + programmatically. diff --git a/docs/source/library-user-guide/index.md b/docs/source/library-user-guide/index.md index fd126a1120ed..76a1b67756f5 100644 --- a/docs/source/library-user-guide/index.md +++ b/docs/source/library-user-guide/index.md @@ -41,3 +41,8 @@ DataFusion is designed to be extensible at all points, including [user guide]: ../user-guide/example-usage.md [contributor guide]: ../contributor-guide/index.md [docs]: https://docs.rs/datafusion/latest/datafusion/#architecture + +```{toctree} +:hidden: +join-preservation +``` diff --git a/docs/source/library-user-guide/join-preservation.md b/docs/source/library-user-guide/join-preservation.md new file mode 100644 index 000000000000..1cc6b1165e6b --- /dev/null +++ b/docs/source/library-user-guide/join-preservation.md @@ -0,0 +1,54 @@ + + +# Join preservation + +Dynamic filter pushdown and other optimizations rely on whether a join preserves +rows from its inputs. The tables below summarise which sides are preserved for +post-join output filtering and for evaluation of `ON`-clause predicates. + +## Output filtering + +| Join type | Left | Right | +| ----------- | ---- | ----- | +| `Inner` | ✓ | ✓ | +| `Left` | ✓ | ✗ | +| `Right` | ✗ | ✓ | +| `Full` | ✗ | ✗ | +| `LeftSemi` | ✓ | ✗ | +| `LeftAnti` | ✓ | ✗ | +| `LeftMark` | ✓ | ✗ | +| `RightSemi` | ✗ | ✓ | +| `RightAnti` | ✗ | ✓ | +| `RightMark` | ✗ | ✓ | + +## `ON`-clause filtering + +| Join type | Left | Right | +| ----------- | ---- | ----- | +| `Inner` | ✓ | ✓ | +| `Left` | ✗ | ✓ | +| `Right` | ✓ | ✗ | +| `Full` | ✗ | ✗ | +| `LeftSemi` | ✓ | ✓ | +| `RightSemi` | ✓ | ✓ | +| `LeftAnti` | ✗ | ✓ | +| `RightAnti` | ✓ | ✗ | +| `LeftMark` | ✗ | ✓ | +| `RightMark` | ✓ | ✗ | diff --git a/docs/source/library-user-guide/upgrading.md b/docs/source/library-user-guide/upgrading.md index e54b4771ab03..d9356c843ff5 100644 --- a/docs/source/library-user-guide/upgrading.md +++ b/docs/source/library-user-guide/upgrading.md @@ -47,9 +47,8 @@ allowing DataFusion to prune the probe side as join keys are discovered at runtime. `HashJoinExec` builds from its left child and probes with its right child (Left = left child, Right = right child). Full joins are not supported and only equi-join keys contribute. Non-equi predicates require range analysis and -cross-conjunct reasoning (future work). Dynamic filters obey -`NullEqualsNothing` semantics, so rows with null join keys never match and -generate no filter values. Consider enabling +cross-conjunct reasoning (future work). Rows with `NULL` join keys do not +produce dynamic filter values (`NullEqualsNothing`). Consider enabling `datafusion.optimizer.filter_null_join_keys` to remove nulls early. This behavior is controlled by the `datafusion.optimizer.enable_dynamic_filter_pushdown` configuration option (on @@ -61,16 +60,26 @@ for Parquet this means enabling not benefit. | JoinType | Probe side pruned | -| ------------------------ | ---------------- | -| `Inner`, `Left` | Right input | -| `Right` | Left input | -| `LeftSemi`, `LeftAnti` | Left input | -| `RightSemi`, `RightAnti` | Right input | +| ------------------------ | ----------------- | +| `Inner`, `Left` | Right input | +| `Right` | Left input | +| `LeftSemi`, `LeftAnti` | Left input | +| `RightSemi`, `RightAnti` | Right input | + +See [join preservation tables](join-preservation.md) for more detail on which +inputs survive each join type. Dynamic filters are most effective when the join keys are highly selective. You can disable the feature by setting `datafusion.optimizer.enable_dynamic_filter_pushdown=false`. +```rust +use datafusion::prelude::SessionConfig; + +let config = SessionConfig::new() + .with_optimizer_enable_dynamic_filter_pushdown(false); +``` + For example: ```sql diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 6ca6091bbf89..dbd787cf3090 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -129,7 +129,7 @@ The following configuration settings are available: | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| `datafusion.optimizer.enable_dynamic_filter_pushdown` | true | Pushes down dynamic filters produced by joins to prune the probe side (`HashJoinExec` builds left, probes right). Only equi-join keys contribute; see the [join-type table](../library-user-guide/upgrading.md#dynamic-filter-pushdown-for-joins). Formats without predicate pushdown (CSV/JSON) see no benefit. Interacts with `execution.parquet.pushdown_filters` and `optimizer.filter_null_join_keys`. +| `datafusion.optimizer.enable_dynamic_filter_pushdown` | true | Pushes down dynamic filters produced by joins[^dfpd] | | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | | datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | @@ -174,6 +174,8 @@ The following configuration settings are available: | datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | | datafusion.format.types_info | false | Show types in visual representation batches | +[^dfpd]: See [Upgrade Guide](../library-user-guide/upgrading.md#dynamic-filter-pushdown-for-joins) for caveats and supported join types. Formats without predicate pushdown (CSV/JSON) see no benefit. Interacts with `execution.parquet.pushdown_filters` and `optimizer.filter_null_join_keys`. + # Runtime Configuration Settings DataFusion runtime configurations can be set via SQL using the `SET` command. From 51bcf4a517009948a882965c64818efce6cf72ec Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 08:38:20 +0800 Subject: [PATCH 38/72] Enhance dynamic filter pushdown tests and improve output clarity. Add `predicate=` to DataSourceExec in multiple tests for better visibility of filter pushdown behavior and update join execution details to include probe side and keys. --- datafusion/core/tests/dataframe/mod.rs | 170 +++++++++--------- .../physical_optimizer/filter_pushdown/mod.rs | 76 ++++---- .../physical_optimizer/projection_pushdown.rs | 4 +- datafusion/core/tests/sql/explain_analyze.rs | 6 +- 4 files changed, 128 insertions(+), 128 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 38dc0dc73569..02c6c73d9a5d 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -2664,23 +2664,23 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { assert_snapshot!( pretty_format_batches(&sql_results).unwrap(), @r" - +---------------+------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __correlated_sq_1 | - | | Projection: count(Int64(1)) AS count(*) | - | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] | - | | TableScan: t2 projection=[] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | - | | ProjectionExec: expr=[4 as count(*)] | - | | PlaceholderRowExec | - | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+------------------------------------------------------------------------------------------------------------------------+ + +---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __correlated_sq_1 | + | | Projection: count(Int64(1)) AS count(*) | + | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] | + | | TableScan: t2 projection=[] | + | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | + | | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1], probe_side=Left, probe_keys=0 | + | | ProjectionExec: expr=[4 as count(*)] | + | | PlaceholderRowExec | + | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+ " ); @@ -2710,22 +2710,22 @@ async fn test_count_wildcard_on_where_in() -> Result<()> { assert_snapshot!( pretty_format_batches(&df_results).unwrap(), @r" - +---------------+------------------------------------------------------------------------------------------------------------------------+ - | plan_type | plan | - +---------------+------------------------------------------------------------------------------------------------------------------------+ - | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | - | | TableScan: t1 projection=[a, b] | - | | SubqueryAlias: __correlated_sq_1 | - | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] | - | | TableScan: t2 projection=[] | - | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1] | - | | ProjectionExec: expr=[4 as count(*)] | - | | PlaceholderRowExec | - | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | | - +---------------+------------------------------------------------------------------------------------------------------------------------+ + +---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+ + | plan_type | plan | + +---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+ + | logical_plan | LeftSemi Join: CAST(t1.a AS Int64) = __correlated_sq_1.count(*) | + | | TableScan: t1 projection=[a, b] | + | | SubqueryAlias: __correlated_sq_1 | + | | Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] | + | | TableScan: t2 projection=[] | + | physical_plan | CoalesceBatchesExec: target_batch_size=8192 | + | | HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(count(*)@0, CAST(t1.a AS Int64)@2)], projection=[a@0, b@1], probe_side=Left, probe_keys=0 | + | | ProjectionExec: expr=[4 as count(*)] | + | | PlaceholderRowExec | + | | ProjectionExec: expr=[a@0 as a, b@1 as b, CAST(a@0 AS Int64) as CAST(t1.a AS Int64)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------+ " ); @@ -2935,32 +2935,32 @@ 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 | CoalesceBatchesExec: target_batch_size=8192 | - | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | 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))] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(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 | CoalesceBatchesExec: target_batch_size=8192 | + | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4], probe_side=Right, probe_keys=0 | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | 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))] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(Int64(1))] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------+ " ); @@ -2992,32 +2992,32 @@ 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 | CoalesceBatchesExec: target_batch_size=8192 | - | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4] | - | | DataSourceExec: partitions=1, partition_sizes=[1] | - | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | - | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | - | | CoalesceBatchesExec: target_batch_size=8192 | - | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | - | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | - | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | - | | 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 | CoalesceBatchesExec: target_batch_size=8192 | + | | FilterExec: CASE WHEN __always_true@3 IS NULL THEN 0 ELSE count(*)@2 END > 0, projection=[a@0, b@1] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@1)], projection=[a@0, b@1, count(*)@2, __always_true@4], probe_side=Right, probe_keys=0 | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | ProjectionExec: expr=[count(*)@1 as count(*), a@0 as a, true as __always_true] | + | | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count(*)] | + | | CoalesceBatchesExec: target_batch_size=8192 | + | | RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 | + | | RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 | + | | AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count(*)] | + | | DataSourceExec: partitions=1, partition_sizes=[1] | + | | | + +---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------+ " ); diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index e5c019f59f5e..2cf1cf65e3e7 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -75,7 +75,7 @@ fn test_pushdown_into_scan() { OptimizationTest: input: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo @@ -110,11 +110,11 @@ fn test_pushdown_volatile_functions_not_allowed() { OptimizationTest: input: - FilterExec: a@0 = random() - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - FilterExec: a@0 = random() - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= ", ); } @@ -137,11 +137,11 @@ fn test_pushdown_into_scan_with_config_options() { OptimizationTest: input: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= " ); @@ -156,7 +156,7 @@ fn test_pushdown_into_scan_with_config_options() { OptimizationTest: input: - FilterExec: a@0 = foo - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo @@ -245,9 +245,9 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { insta::assert_snapshot!( format_plan_for_test(&plan), @r" - - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter_keys=0 + - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false] - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] " ); @@ -270,7 +270,7 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { @r" - SortExec: TopK(fetch=2), expr=[e@4 ASC], preserve_partitioning=[false], filter=[e@4 IS NULL OR e@4 < bb], filter_keys=2 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_filter=[d@0 >= aa AND d@0 <= ab], probe_side=Right, probe_keys=2 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= aa AND d@0 <= ab ] AND DynamicFilterPhysicalExpr [ e@1 IS NULL OR e@1 < bb ] " ); @@ -371,13 +371,13 @@ async fn test_static_filter_pushdown_through_hash_join() { - FilterExec: a@0 = d@3 - FilterExec: e@4 = ba - FilterExec: a@0 = aa - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate= output: Ok: - FilterExec: a@0 = d@3 - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = aa - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=e@1 = ba " @@ -417,14 +417,14 @@ async fn test_static_filter_pushdown_through_hash_join() { OptimizationTest: input: - FilterExec: a@0 = aa - - HashJoinExec: mode=Partitioned, join_type=Left, on=[(a@0, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true + - HashJoinExec: mode=Partitioned, join_type=Left, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate= output: Ok: - - HashJoinExec: mode=Partitioned, join_type=Left, on=[(a@0, d@0)] + - HashJoinExec: mode=Partitioned, join_type=Left, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = aa - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate= " ); } @@ -445,7 +445,7 @@ fn test_filter_collapse() { input: - FilterExec: b@1 = bar - FilterExec: a@0 = foo - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo AND b@1 = bar @@ -472,7 +472,7 @@ fn test_filter_with_projection() { OptimizationTest: input: - FilterExec: a@0 = foo, projection=[b@1, a@0] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - ProjectionExec: expr=[b@1 as b, a@0 as a] @@ -495,7 +495,7 @@ fn test_filter_with_projection() { OptimizationTest: input: - FilterExec: a@0 = foo, projection=[b@1] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - ProjectionExec: expr=[b@1 as b] @@ -592,7 +592,7 @@ fn test_no_pushdown_through_aggregates() { - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt], ordering_mode=PartiallySorted([0]) - FilterExec: a@0 = foo - CoalesceBatchesExec: target_batch_size=10 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - FilterExec: b@1 = bar @@ -619,7 +619,7 @@ fn test_node_handles_child_pushdown_result() { OptimizationTest: input: - TestInsertExec { inject_filter: true } - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - TestInsertExec { inject_filter: true } @@ -638,12 +638,12 @@ fn test_node_handles_child_pushdown_result() { OptimizationTest: input: - TestInsertExec { inject_filter: true } - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate= output: Ok: - TestInsertExec { inject_filter: false } - FilterExec: a@0 = foo - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate= ", ); @@ -706,7 +706,7 @@ async fn test_topk_dynamic_filter_pushdown() { OptimizationTest: input: - SortExec: TopK(fetch=1), expr=[b@1 DESC NULLS LAST], preserve_partitioning=[false] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - SortExec: TopK(fetch=1), expr=[b@1 DESC NULLS LAST], preserve_partitioning=[false] @@ -811,12 +811,12 @@ async fn test_hashjoin_dynamic_filter_pushdown() { @r" OptimizationTest: input: - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate= output: Ok: - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] ", @@ -1145,16 +1145,16 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { @r" OptimizationTest: input: - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], probe_side=Right, probe_keys=0 + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true, predicate= + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_side=Right, probe_keys=0 + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate= + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate= output: Ok: - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], probe_side=Right, probe_keys=0 + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true, predicate= + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] ", @@ -1184,8 +1184,8 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { format!("{}", format_plan_for_test(&plan)), @r" - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], probe_filter=[b@0 >= aa AND b@0 <= ab], probe_side=Right, probe_keys=2 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_filter=[d@0 >= ca AND d@0 <= ce], probe_side=Right, probe_keys=2 + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true, predicate= + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_filter=[d@0 >= ca AND d@0 <= ce], probe_side=Right, probe_keys=5 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ b@0 >= aa AND b@0 <= ab ] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= ca AND d@0 <= ce ] " @@ -1290,13 +1290,13 @@ async fn test_hashjoin_parent_filter_pushdown() { - FilterExec: a@0 = d@3 - FilterExec: e@4 = ba - FilterExec: a@0 = aa - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate= output: Ok: - FilterExec: a@0 = d@3 - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)] + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = aa - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=e@1 = ba " diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 5d396b6b0247..e66e4941edb4 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -1281,7 +1281,7 @@ fn test_hash_join_after_projection() -> Result<()> { actual, @r" ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@7 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 + 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, probe_side=Right, probe_keys=0 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 " @@ -1299,7 +1299,7 @@ fn test_hash_join_after_projection() -> Result<()> { 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] + 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], probe_side=Right, probe_keys=0 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 " diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 852b350b27df..ca2504f79778 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -651,9 +651,9 @@ async fn test_physical_plan_display_indent_multi_children() { assert_snapshot!( actual, - @r###" + @r" CoalesceBatchesExec: target_batch_size=4096 - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@0] + HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@0], probe_side=Right, probe_keys=0 CoalesceBatchesExec: target_batch_size=4096 RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000 RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1 @@ -663,7 +663,7 @@ async fn test_physical_plan_display_indent_multi_children() { RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1 ProjectionExec: expr=[c1@0 as c2] DataSourceExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], file_type=csv, has_header=true - "### + " ); } From 4ceb8ddf8350492722597380237982b6b5c56d21 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 08:48:30 +0800 Subject: [PATCH 39/72] Enhance join execution details in tests. Update expected output to include probe side and probe keys for better clarity in join operations. --- .../tests/physical_optimizer/enforce_distribution.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index fd847763124a..ba35ae06453e 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -859,7 +859,7 @@ fn join_after_agg_alias() -> Result<()> { // Only two RepartitionExecs added let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)], probe_side=Right, probe_keys=0", " AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", " AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", @@ -912,7 +912,7 @@ fn hash_join_key_ordering() -> Result<()> { // Only two RepartitionExecs added let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)], probe_side=Right, probe_keys=0", " ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", " AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", " RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", @@ -1033,9 +1033,9 @@ fn multi_hash_join_key_ordering() -> Result<()> { // The bottom joins' join key ordering is adjusted based on the top join. And the top join should not introduce additional RepartitionExec let expected = &[ "FilterExec: c@6 > 1", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)], probe_side=Right, probe_keys=0", " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", @@ -1043,7 +1043,7 @@ fn multi_hash_join_key_ordering() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", From 10c2171b6dc2c964eb880100cbe346dcbd212c3d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 09:08:45 +0800 Subject: [PATCH 40/72] Enhance multi_hash_joins test output. Include probe side and probe keys in join plan formatting for improved clarity. --- .../core/tests/physical_optimizer/enforce_distribution.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index ba35ae06453e..ef65e2d8fb2b 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -562,7 +562,7 @@ fn multi_hash_joins() -> Result<()> { for join_type in join_types { let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); let join_plan = |shift| -> String { - format!("{}HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]", " ".repeat(shift)) + format!("{}HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)], probe_side=Right, probe_keys=0", " ".repeat(shift)) }; let join_plan_indent2 = join_plan(2); let join_plan_indent4 = join_plan(4); @@ -587,7 +587,7 @@ fn multi_hash_joins() -> Result<()> { &join_type, ); let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]"); + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)], probe_side=Right, probe_keys=0"); let expected = match join_type { // Should include 3 RepartitionExecs From 02596fcbef6b0b9b2efc748208d2871959b0e5eb Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 09:17:19 +0800 Subject: [PATCH 41/72] Fix probe side and keys in multi_hash_joins test output. Adjust formatting to correctly reflect probe side based on join type for improved clarity. --- .../tests/physical_optimizer/enforce_distribution.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index ef65e2d8fb2b..3a511241eb40 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -586,8 +586,11 @@ fn multi_hash_joins() -> Result<()> { &top_join_on, &join_type, ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)], probe_side=Right, probe_keys=0"); + let top_join_plan = if join_type == JoinType::Right { + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)], probe_side=Left, probe_keys=0") + } else { + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)], probe_side=Right, probe_keys=0") + }; let expected = match join_type { // Should include 3 RepartitionExecs @@ -651,7 +654,7 @@ fn multi_hash_joins() -> Result<()> { JoinType::RightSemi | JoinType::RightAnti => format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)]"), _ => - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)]"), + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)], probe_side=Right, probe_keys=0"), }; let expected = match join_type { From 24f6367703a377c0dc3b14f68351bfb984cc29f1 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 09:28:35 +0800 Subject: [PATCH 42/72] Fix probe side determination in multi_hash_joins test output. Adjust formatting to correctly reflect probe side based on join type for improved clarity. --- .../.projection_pushdown.rs.pending-snap | 466 ++++++++++++++++++ .../enforce_distribution.rs | 64 ++- .../filter_pushdown/.mod.rs.pending-snap | 355 +++++++++++++ 3 files changed, 868 insertions(+), 17 deletions(-) create mode 100644 datafusion/core/tests/physical_optimizer/.projection_pushdown.rs.pending-snap create mode 100644 datafusion/core/tests/physical_optimizer/filter_pushdown/.mod.rs.pending-snap diff --git a/datafusion/core/tests/physical_optimizer/.projection_pushdown.rs.pending-snap b/datafusion/core/tests/physical_optimizer/.projection_pushdown.rs.pending-snap new file mode 100644 index 000000000000..8dbadb69b082 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/.projection_pushdown.rs.pending-snap @@ -0,0 +1,466 @@ +{"run_id":"1755131915-417051000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755132529-968823000","line":824,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":841,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":435,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":451,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":882,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":899,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1280,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1298,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755132529-968823000","line":978,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":996,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1108,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1126,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":473,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":489,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1186,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1202,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":733,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":750,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1615,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1661,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":665,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":682,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1362,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1378,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1433,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1449,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1487,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1503,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1530,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1548,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":824,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":841,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":435,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":451,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":882,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":899,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1280,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1298,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":978,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":996,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1108,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1126,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":473,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":489,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1186,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1202,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":733,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":750,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1615,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1661,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":665,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":682,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1362,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1378,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1433,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1487,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1449,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1503,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1530,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1548,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755133744-602671000","line":824,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":841,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":435,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":451,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":882,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":899,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1280,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1298,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":978,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":996,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1108,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1126,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":473,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":489,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1186,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1202,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":733,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":750,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1615,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1661,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":665,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":682,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1362,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1378,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1433,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1449,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1487,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1503,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1530,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1548,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755133849-534965000","line":824,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":841,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":435,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":451,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":882,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":899,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1280,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1298,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":978,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":996,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1108,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1126,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":473,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":489,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1186,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1202,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":733,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":750,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1615,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1661,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":665,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":682,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1362,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1378,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1433,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1449,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1487,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1503,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1530,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1548,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755134637-845146000","line":824,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":841,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":435,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":451,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":882,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":899,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1280,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1298,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":978,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":996,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1108,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1126,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":473,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":489,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1186,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1202,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":733,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":750,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1615,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1661,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":665,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":682,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1362,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1378,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1433,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1449,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1487,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1503,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1530,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1548,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755134931-99889000","line":824,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":841,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":435,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":451,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":882,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":899,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1280,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1298,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":978,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":996,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1108,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1126,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":473,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":489,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1186,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1202,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":733,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":750,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1615,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1661,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":665,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":682,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1362,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1378,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1433,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1449,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1487,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1503,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1530,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1548,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755135191-689174000","line":824,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":841,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":435,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":451,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":882,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":899,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1280,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":978,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1298,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":996,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1108,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1126,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":473,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":489,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1186,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1202,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":733,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":750,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1615,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1661,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":665,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":682,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1362,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1378,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1433,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1449,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1487,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1503,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1530,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1548,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755136115-607436000","line":824,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":841,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":435,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":451,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":882,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":899,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1280,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1298,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":978,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":996,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1108,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1126,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":473,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":489,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1186,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1202,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":733,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":750,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1615,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1661,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1362,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1378,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1433,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":665,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1449,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":682,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1487,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1503,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1530,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1548,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755136339-714449000","line":824,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":841,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":435,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":451,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":882,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1280,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1298,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":899,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":978,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":996,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1108,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1126,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":473,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":489,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1186,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1202,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":733,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":750,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1615,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1661,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":665,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":682,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1362,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1378,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1433,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1449,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1487,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1503,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1530,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1548,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755136792-165744000","line":824,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":841,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":435,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":451,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":882,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":899,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1280,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":978,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":996,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1298,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1108,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1126,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":473,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":489,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1186,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1202,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":733,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":750,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1615,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1661,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":665,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":682,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1362,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1378,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1433,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1449,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1487,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1503,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1530,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1548,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755137109-520840000","line":824,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":841,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":435,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":451,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":882,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":899,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1280,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1298,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":978,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":996,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1108,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1126,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":473,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":489,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1186,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1202,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":733,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":750,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1615,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1661,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":665,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":682,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1362,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1378,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1433,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1449,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1487,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1503,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1530,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1548,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755137323-682556000","line":824,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":841,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":435,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":451,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":882,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":899,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1280,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1298,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":978,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":996,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1108,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1126,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":473,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":489,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1186,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1202,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":733,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":750,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1615,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1661,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":665,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":682,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1362,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1378,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1433,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1449,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1487,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1503,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1530,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1548,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755137528-237690000","line":824,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":841,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":435,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":451,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":882,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":899,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1280,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1298,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":978,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":996,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1108,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1126,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":473,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":489,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1186,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1202,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":733,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":750,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1615,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1661,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":665,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":682,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1362,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1378,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1433,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1449,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1487,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1503,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1530,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1548,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755137883-460671000","line":824,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":435,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":451,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":841,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":882,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1280,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":899,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1298,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":978,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":996,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1108,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1126,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":473,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":489,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1186,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1202,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":733,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":750,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1615,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1661,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":665,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":682,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1362,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1378,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1433,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1449,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1487,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1503,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1530,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1548,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} +{"run_id":"1755137982-357195000","line":824,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":841,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":435,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":451,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":882,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":899,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1280,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":978,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1298,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":996,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1108,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1126,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":473,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":489,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1186,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1202,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":733,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":750,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1615,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1661,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":665,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":682,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1362,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1378,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1433,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1449,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1487,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1530,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1548,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1503,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 3a511241eb40..90e50266521a 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -562,7 +562,25 @@ fn multi_hash_joins() -> Result<()> { for join_type in join_types { let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); let join_plan = |shift| -> String { - format!("{}HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)], probe_side=Right, probe_keys=0", " ".repeat(shift)) + if join_type == JoinType::Right + || join_type == JoinType::RightSemi + || join_type == JoinType::RightAnti + { + format!( + "{}HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)], probe_side=Left, probe_keys=0", + " ".repeat(shift) + ) + } else if join_type == JoinType::Full { + format!( + "{}HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]", + " ".repeat(shift) + ) + } else { + format!( + "{}HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)], probe_side=Right, probe_keys=0", + " ".repeat(shift) + ) + } }; let join_plan_indent2 = join_plan(2); let join_plan_indent4 = join_plan(4); @@ -587,9 +605,17 @@ fn multi_hash_joins() -> Result<()> { &join_type, ); let top_join_plan = if join_type == JoinType::Right { - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)], probe_side=Left, probe_keys=0") + format!( + "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)], probe_side=Left, probe_keys=0" + ) + } else if join_type == JoinType::Full { + format!( + "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]" + ) } else { - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)], probe_side=Right, probe_keys=0") + format!( + "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)], probe_side=Right, probe_keys=0" + ) }; let expected = match join_type { @@ -651,8 +677,12 @@ fn multi_hash_joins() -> Result<()> { let top_join = hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); let top_join_plan = match join_type { - JoinType::RightSemi | JoinType::RightAnti => - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)]"), + JoinType::Right => + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)], probe_side=Left, probe_keys=0"), + JoinType::RightSemi | JoinType::RightAnti=> + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)], probe_side=Left, probe_keys=0"), + JoinType::Full => + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)]"), _ => format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)], probe_side=Right, probe_keys=0"), }; @@ -738,9 +768,9 @@ fn multi_joins_after_alias() -> Result<()> { // Output partition need to respect the Alias and should not introduce additional RepartitionExec let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)], probe_side=Right, probe_keys=0", " ProjectionExec: expr=[a@0 as a1, a@0 as a2]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", @@ -765,9 +795,9 @@ fn multi_joins_after_alias() -> Result<()> { // Output partition need to respect the Alias and should not introduce additional RepartitionExec let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)], probe_side=Right, probe_keys=0", " ProjectionExec: expr=[a@0 as a1, a@0 as a2]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", @@ -817,11 +847,11 @@ fn multi_joins_after_multi_alias() -> Result<()> { // The Column 'a' has different meaning now after the two Projections // The original Output partition can not satisfy the Join requirements and need to add an additional RepartitionExec let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " ProjectionExec: expr=[c1@0 as a]", " ProjectionExec: expr=[c@2 as c1]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", @@ -1169,7 +1199,7 @@ fn reorder_join_keys_to_left_input() -> Result<()> { &join_type, ); let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type); + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)], probe_side=Left, probe_keys=0", &join_type); let reordered = reorder_join_keys_to_inputs(top_join)?; @@ -1177,7 +1207,7 @@ fn reorder_join_keys_to_left_input() -> Result<()> { let expected = &[ top_join_plan.as_str(), " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", @@ -1185,7 +1215,7 @@ fn reorder_join_keys_to_left_input() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", @@ -1303,7 +1333,7 @@ fn reorder_join_keys_to_right_input() -> Result<()> { &join_type, ); let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)]", &join_type); + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)], probe_side=Right, probe_keys=0", &join_type); let reordered = reorder_join_keys_to_inputs(top_join)?; @@ -1311,7 +1341,7 @@ fn reorder_join_keys_to_right_input() -> Result<()> { let expected = &[ top_join_plan.as_str(), " ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", @@ -1319,7 +1349,7 @@ fn reorder_join_keys_to_right_input() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)], probe_side=Right, probe_keys=0", " RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/.mod.rs.pending-snap b/datafusion/core/tests/physical_optimizer/filter_pushdown/.mod.rs.pending-snap new file mode 100644 index 000000000000..c9bd592f79c8 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/.mod.rs.pending-snap @@ -0,0 +1,355 @@ +{"run_id":"1755131915-417051000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755131915-417051000","line":521,"new":null,"old":null} +{"run_id":"1755131915-417051000","line":72,"new":null,"old":null} +{"run_id":"1755131915-417051000","line":130,"new":null,"old":null} +{"run_id":"1755131915-417051000","line":149,"new":null,"old":null} +{"run_id":"1755131915-417051000","line":107,"new":null,"old":null} +{"run_id":"1755131915-417051000","line":366,"new":null,"old":null} +{"run_id":"1755131915-417051000","line":414,"new":null,"old":null} +{"run_id":"1755131915-417051000","line":703,"new":null,"old":null} +{"run_id":"1755131915-417051000","line":735,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":245,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":268,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":441,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":469,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":492,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":809,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":845,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1285,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1143,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":1183,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":585,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":616,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":635,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":521,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":72,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":130,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":149,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":107,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":366,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":414,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":703,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":735,"new":null,"old":null} +{"run_id":"1755132529-968823000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755133350-769504000","line":245,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":268,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":441,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":469,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":492,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":809,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":845,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1285,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1143,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":1183,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":585,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":616,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":635,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":521,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":72,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":130,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":149,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":107,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":366,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":414,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":703,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":735,"new":null,"old":null} +{"run_id":"1755133350-769504000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755133744-602671000","line":245,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":268,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":441,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":469,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":492,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":809,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":845,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1285,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1143,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":1183,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":585,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":616,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":635,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755133744-602671000","line":521,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":72,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":130,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":149,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":107,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":366,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":414,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":703,"new":null,"old":null} +{"run_id":"1755133744-602671000","line":735,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":245,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":268,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":441,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":469,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":492,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":809,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":845,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1285,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1143,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":1183,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":585,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":616,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":635,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755133849-534965000","line":521,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":72,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":130,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":149,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":107,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":366,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":414,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":703,"new":null,"old":null} +{"run_id":"1755133849-534965000","line":735,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":245,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":268,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":441,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":469,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":492,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":809,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":845,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1285,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1143,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":1183,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":585,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":616,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":635,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":521,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":72,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":130,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":149,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":107,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":366,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":414,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":703,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":735,"new":null,"old":null} +{"run_id":"1755134637-845146000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755134931-99889000","line":245,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":268,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":441,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":469,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":492,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":809,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":845,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1285,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1143,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":1183,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":585,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":616,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":635,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":521,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":72,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":130,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":149,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":107,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":366,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":414,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":703,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":735,"new":null,"old":null} +{"run_id":"1755134931-99889000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755135191-689174000","line":245,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":268,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":441,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":469,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":492,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":809,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":845,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1285,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1143,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":1183,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":585,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":616,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":635,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755135191-689174000","line":521,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":72,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":130,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":149,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":107,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":366,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":414,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":703,"new":null,"old":null} +{"run_id":"1755135191-689174000","line":735,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":245,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":268,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":441,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":469,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":492,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":809,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":845,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1285,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1143,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":1183,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":585,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":616,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":635,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":521,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":72,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":130,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":149,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":107,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":366,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":414,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":703,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":735,"new":null,"old":null} +{"run_id":"1755136115-607436000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755136339-714449000","line":245,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":268,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":441,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":469,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":492,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":809,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":845,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1285,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1143,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":1183,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":585,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":616,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":635,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755136339-714449000","line":521,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":72,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":130,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":149,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":107,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":366,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":414,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":703,"new":null,"old":null} +{"run_id":"1755136339-714449000","line":735,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":245,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":268,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":441,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":469,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":492,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":809,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":845,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1285,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1143,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":1183,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":585,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":616,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":635,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":521,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":72,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":130,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":149,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":107,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755136792-165744000","line":366,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":703,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":414,"new":null,"old":null} +{"run_id":"1755136792-165744000","line":735,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":245,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":268,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":441,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":469,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":492,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":809,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":845,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1285,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1143,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":1183,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":585,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":616,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":635,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755137109-520840000","line":521,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":72,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":130,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":149,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":107,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":366,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":414,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":703,"new":null,"old":null} +{"run_id":"1755137109-520840000","line":735,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":245,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":268,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":441,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":469,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":492,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":809,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":845,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1285,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1143,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":1183,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":585,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":616,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":635,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":521,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":72,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":130,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":149,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":107,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":366,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":414,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":703,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":735,"new":null,"old":null} +{"run_id":"1755137323-682556000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755137528-237690000","line":245,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":268,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":441,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":469,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":492,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":809,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":845,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1285,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1143,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":1183,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":585,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":616,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":635,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755137528-237690000","line":521,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":72,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":130,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":149,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":107,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":366,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":414,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":703,"new":null,"old":null} +{"run_id":"1755137528-237690000","line":735,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":245,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":268,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":441,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":469,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":492,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":809,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":845,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1285,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1143,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":1183,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":585,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":616,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":635,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} +{"run_id":"1755137883-460671000","line":521,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":72,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":130,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":149,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":107,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":366,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":414,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":703,"new":null,"old":null} +{"run_id":"1755137883-460671000","line":735,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":245,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":268,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":441,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":469,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":492,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":809,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":845,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1285,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1143,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":1183,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":585,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":616,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":635,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":521,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":72,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":130,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":149,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":107,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":366,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":414,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":703,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":735,"new":null,"old":null} +{"run_id":"1755137982-357195000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} From 0b1fc311b0f8bce1ee5f665a1857e23b87173975 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 10:30:25 +0800 Subject: [PATCH 43/72] Implement snap changes to enhance functionality and improve performance --- .../.projection_pushdown.rs.pending-snap | 466 ------------------ .../filter_pushdown/.mod.rs.pending-snap | 355 ------------- 2 files changed, 821 deletions(-) delete mode 100644 datafusion/core/tests/physical_optimizer/.projection_pushdown.rs.pending-snap delete mode 100644 datafusion/core/tests/physical_optimizer/filter_pushdown/.mod.rs.pending-snap diff --git a/datafusion/core/tests/physical_optimizer/.projection_pushdown.rs.pending-snap b/datafusion/core/tests/physical_optimizer/.projection_pushdown.rs.pending-snap deleted file mode 100644 index 8dbadb69b082..000000000000 --- a/datafusion/core/tests/physical_optimizer/.projection_pushdown.rs.pending-snap +++ /dev/null @@ -1,466 +0,0 @@ -{"run_id":"1755131915-417051000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755132529-968823000","line":824,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":841,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":435,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":451,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":882,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":899,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1280,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1298,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755132529-968823000","line":978,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":996,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1108,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1126,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":473,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":489,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1186,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1202,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":733,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":750,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1615,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1661,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":665,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":682,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1362,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1378,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1433,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1449,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1487,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1503,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1530,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1548,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":824,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":841,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":435,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":451,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":882,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":899,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1280,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1298,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":978,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":996,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1108,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1126,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":473,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":489,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1186,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1202,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":733,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":750,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1615,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1661,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":665,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":682,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1362,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1378,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1433,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1487,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1449,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1503,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1530,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1548,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755133744-602671000","line":824,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":841,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":435,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":451,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":882,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":899,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1280,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1298,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":978,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":996,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1108,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1126,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":473,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":489,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1186,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1202,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":733,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":750,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1615,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1661,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":665,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":682,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1362,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1378,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1433,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1449,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1487,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1503,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1530,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1548,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755133849-534965000","line":824,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":841,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":435,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":451,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":882,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":899,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1280,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1298,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":978,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":996,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1108,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1126,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":473,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":489,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1186,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1202,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":733,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":750,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1615,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1661,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":665,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":682,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1362,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1378,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1433,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1449,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1487,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1503,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1530,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1548,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755134637-845146000","line":824,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":841,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":435,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":451,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":882,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":899,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1280,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1298,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":978,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":996,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1108,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1126,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":473,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":489,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1186,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1202,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":733,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":750,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1615,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1661,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":665,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":682,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1362,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1378,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1433,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1449,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1487,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1503,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1530,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1548,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755134931-99889000","line":824,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":841,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":435,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":451,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":882,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":899,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1280,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1298,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":978,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":996,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1108,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1126,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":473,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":489,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1186,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1202,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":733,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":750,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1615,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1661,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":665,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":682,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1362,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1378,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1433,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1449,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1487,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1503,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1530,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1548,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755135191-689174000","line":824,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":841,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":435,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":451,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":882,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":899,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1280,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":978,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1298,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":996,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1108,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1126,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":473,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":489,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1186,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1202,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":733,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":750,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1615,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1661,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":665,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":682,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1362,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1378,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1433,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1449,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1487,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1503,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1530,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1548,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755136115-607436000","line":824,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":841,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":435,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":451,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":882,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":899,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1280,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1298,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":978,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":996,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1108,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1126,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":473,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":489,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1186,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1202,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":733,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":750,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1615,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1661,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1362,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1378,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1433,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":665,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1449,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":682,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1487,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1503,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1530,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1548,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755136339-714449000","line":824,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":841,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":435,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":451,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":882,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1280,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1298,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":899,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":978,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":996,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1108,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1126,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":473,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":489,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1186,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1202,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":733,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":750,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1615,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1661,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":665,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":682,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1362,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1378,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1433,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1449,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1487,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1503,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1530,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1548,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755136792-165744000","line":824,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":841,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":435,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":451,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":882,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":899,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1280,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":978,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":996,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1298,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1108,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1126,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":473,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":489,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1186,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1202,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":733,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":750,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1615,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1661,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":665,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":682,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1362,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1378,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1433,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1449,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1487,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1503,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1530,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1548,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755137109-520840000","line":824,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":841,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":435,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":451,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":882,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":899,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1280,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1298,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":978,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":996,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1108,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1126,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":473,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":489,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1186,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1202,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":733,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":750,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1615,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1661,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":665,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":682,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1362,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1378,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1433,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1449,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1487,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1503,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1530,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1548,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755137323-682556000","line":824,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":841,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":435,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":451,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":882,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":899,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1280,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1298,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":978,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":996,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1108,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1126,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":473,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":489,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1186,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1202,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":733,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":750,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1615,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1661,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":665,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":682,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1362,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1378,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1433,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1449,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1487,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1503,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1530,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1548,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755137528-237690000","line":824,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":841,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":435,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":451,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":882,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":899,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1280,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1298,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":978,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":996,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1108,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1126,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":473,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":489,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1186,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1202,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":733,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":750,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1615,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1661,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":665,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":682,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1362,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1378,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1433,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1449,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1487,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1503,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1530,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1548,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755137883-460671000","line":824,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":435,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":451,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":841,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":882,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1280,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":899,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1298,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":978,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":996,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1108,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1126,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":473,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":489,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1186,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1202,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":733,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":750,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1615,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1661,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":665,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":682,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1362,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1378,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1433,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1449,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1487,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1503,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1530,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1548,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} -{"run_id":"1755137982-357195000","line":824,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":841,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":435,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":451,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":882,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":899,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1280,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":978,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1298,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":996,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1108,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1126,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":473,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":489,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1186,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1202,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":733,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":750,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1615,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1661,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":665,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":682,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1362,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1378,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1433,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1449,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1487,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1530,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1548,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1503,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1326,"new":{"module_name":"core_integration__physical_optimizer__projection_pushdown","snapshot_name":"hash_join_after_projection-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/projection_pushdown.rs","assertion_line":1326,"expression":"actual"},"snapshot":"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], probe_side=Right, probe_keys=0\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"},"old":{"module_name":"core_integration__physical_optimizer__projection_pushdown","metadata":{},"snapshot":"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]\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false\n DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false"}} diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/.mod.rs.pending-snap b/datafusion/core/tests/physical_optimizer/filter_pushdown/.mod.rs.pending-snap deleted file mode 100644 index c9bd592f79c8..000000000000 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/.mod.rs.pending-snap +++ /dev/null @@ -1,355 +0,0 @@ -{"run_id":"1755131915-417051000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755131915-417051000","line":521,"new":null,"old":null} -{"run_id":"1755131915-417051000","line":72,"new":null,"old":null} -{"run_id":"1755131915-417051000","line":130,"new":null,"old":null} -{"run_id":"1755131915-417051000","line":149,"new":null,"old":null} -{"run_id":"1755131915-417051000","line":107,"new":null,"old":null} -{"run_id":"1755131915-417051000","line":366,"new":null,"old":null} -{"run_id":"1755131915-417051000","line":414,"new":null,"old":null} -{"run_id":"1755131915-417051000","line":703,"new":null,"old":null} -{"run_id":"1755131915-417051000","line":735,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":245,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":268,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":441,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":469,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":492,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":809,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":845,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1285,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1143,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":1183,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":585,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":616,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":635,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":521,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":72,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":130,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":149,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":107,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":366,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":414,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":703,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":735,"new":null,"old":null} -{"run_id":"1755132529-968823000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755133350-769504000","line":245,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":268,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":441,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":469,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":492,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":809,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":845,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1285,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1143,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":1183,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":585,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":616,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":635,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":521,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":72,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":130,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":149,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":107,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":366,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":414,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":703,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":735,"new":null,"old":null} -{"run_id":"1755133350-769504000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755133744-602671000","line":245,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":268,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":441,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":469,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":492,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":809,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":845,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1285,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1143,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":1183,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":585,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":616,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":635,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755133744-602671000","line":521,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":72,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":130,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":149,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":107,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":366,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":414,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":703,"new":null,"old":null} -{"run_id":"1755133744-602671000","line":735,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":245,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":268,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":441,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":469,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":492,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":809,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":845,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1285,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1143,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":1183,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":585,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":616,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":635,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755133849-534965000","line":521,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":72,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":130,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":149,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":107,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":366,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":414,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":703,"new":null,"old":null} -{"run_id":"1755133849-534965000","line":735,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":245,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":268,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":441,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":469,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":492,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":809,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":845,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1285,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1143,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":1183,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":585,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":616,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":635,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":521,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":72,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":130,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":149,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":107,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":366,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":414,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":703,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":735,"new":null,"old":null} -{"run_id":"1755134637-845146000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755134931-99889000","line":245,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":268,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":441,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":469,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":492,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":809,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":845,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1285,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1143,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":1183,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":585,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":616,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":635,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":521,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":72,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":130,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":149,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":107,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":366,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":414,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":703,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":735,"new":null,"old":null} -{"run_id":"1755134931-99889000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755135191-689174000","line":245,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":268,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":441,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":469,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":492,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":809,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":845,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1285,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1143,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":1183,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":585,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":616,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":635,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755135191-689174000","line":521,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":72,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":130,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":149,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":107,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":366,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":414,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":703,"new":null,"old":null} -{"run_id":"1755135191-689174000","line":735,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":245,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":268,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":441,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":469,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":492,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":809,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":845,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1285,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1143,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":1183,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":585,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":616,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":635,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":521,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":72,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":130,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":149,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":107,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":366,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":414,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":703,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":735,"new":null,"old":null} -{"run_id":"1755136115-607436000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755136339-714449000","line":245,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":268,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":441,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":469,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":492,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":809,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":845,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1285,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1143,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":1183,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":585,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":616,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":635,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755136339-714449000","line":521,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":72,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":130,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":149,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":107,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":366,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":414,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":703,"new":null,"old":null} -{"run_id":"1755136339-714449000","line":735,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":245,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":268,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":441,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":469,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":492,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":809,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":845,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1285,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1143,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":1183,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":585,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":616,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":635,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":521,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":72,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":130,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":149,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":107,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755136792-165744000","line":366,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":703,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":414,"new":null,"old":null} -{"run_id":"1755136792-165744000","line":735,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":245,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":268,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":441,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":469,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":492,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":809,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":845,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1285,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1143,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":1183,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":585,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":616,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":635,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755137109-520840000","line":521,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":72,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":130,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":149,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":107,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":366,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":414,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":703,"new":null,"old":null} -{"run_id":"1755137109-520840000","line":735,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":245,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":268,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":441,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":469,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":492,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":809,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":845,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1285,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1143,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":1183,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":585,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":616,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":635,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":521,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":72,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":130,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":149,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":107,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":366,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":414,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":703,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":735,"new":null,"old":null} -{"run_id":"1755137323-682556000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755137528-237690000","line":245,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":268,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":441,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":469,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":492,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":809,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":845,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1285,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1143,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":1183,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":585,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":616,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":635,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755137528-237690000","line":521,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":72,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":130,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":149,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":107,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":366,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":414,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":703,"new":null,"old":null} -{"run_id":"1755137528-237690000","line":735,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":245,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":268,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":441,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":469,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":492,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":809,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":845,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1285,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1143,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":1183,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":585,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":616,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":635,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} -{"run_id":"1755137883-460671000","line":521,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":72,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":130,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":149,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":107,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":366,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":414,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":703,"new":null,"old":null} -{"run_id":"1755137883-460671000","line":735,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":245,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":268,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":441,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":469,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":492,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":809,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":845,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1285,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1143,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":1183,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":585,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":616,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":635,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":521,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":72,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":130,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":149,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":107,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":366,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":414,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":703,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":735,"new":null,"old":null} -{"run_id":"1755137982-357195000","line":655,"new":{"module_name":"core_integration__physical_optimizer__filter_pushdown","snapshot_name":"node_handles_child_pushdown_result-3","metadata":{"source":"datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs","assertion_line":655,"expression":"OptimizationTest::new(plan, FilterPushdown::new(), true)"},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate=\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate="},"old":{"module_name":"core_integration__physical_optimizer__filter_pushdown","metadata":{},"snapshot":"OptimizationTest:\n input:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false\n output:\n Ok:\n - TestInsertExec { inject_filter: false }\n - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false"}} From 910a7ed2e245c74fce8029fb600121e83accacfe Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 10:30:45 +0800 Subject: [PATCH 44/72] Enhance test output for filter pushdown and projection pushdown. Include predicate in DataSourceExec and add probe side and keys in HashJoinExec for improved clarity. --- .../core/tests/physical_optimizer/filter_pushdown/mod.rs | 4 ++-- .../core/tests/physical_optimizer/projection_pushdown.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 2cf1cf65e3e7..af258c467eb4 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -658,11 +658,11 @@ fn test_node_handles_child_pushdown_result() { OptimizationTest: input: - TestInsertExec { inject_filter: false } - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate= output: Ok: - TestInsertExec { inject_filter: false } - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=false, predicate= ", ); } diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index e66e4941edb4..f0709a96c6a8 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -1326,7 +1326,7 @@ fn test_hash_join_after_projection() -> Result<()> { assert_snapshot!( actual, @r" - 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] + 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], probe_side=Right, probe_keys=0 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 f64d0724f253b366ff6aa11e981109d6d6f4110c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 10:30:56 +0800 Subject: [PATCH 45/72] Fix probe side determination in HashJoinExec output. Adjust formatting to reflect correct probe side based on join type. --- .../enforce_distribution.rs | 27 ++++++++++++++++--- 1 file changed, 23 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 90e50266521a..4a4ee749963e 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1198,8 +1198,18 @@ fn reorder_join_keys_to_left_input() -> Result<()> { &top_join_on, &join_type, ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)], probe_side=Left, probe_keys=0", &join_type); + let top_join_plan = if join_type == JoinType::Inner + || join_type == JoinType::Left + || join_type == JoinType::LeftAnti + { + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)], probe_side=Right, probe_keys=0", &join_type) + } else if join_type == JoinType::LeftSemi { + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)], probe_side=Right, probe_keys=0", &join_type) + } else if join_type == JoinType::Full { + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type) + } else { + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)], probe_side=Left, probe_keys=0", &join_type) + }; let reordered = reorder_join_keys_to_inputs(top_join)?; @@ -1332,8 +1342,17 @@ fn reorder_join_keys_to_right_input() -> Result<()> { &top_join_on, &join_type, ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)], probe_side=Right, probe_keys=0", &join_type); + let top_join_plan = if join_type == JoinType::Inner + || join_type == JoinType::Left + || join_type == JoinType::LeftSemi + || join_type == JoinType::LeftAnti + { + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)], probe_side=Right, probe_keys=0", &join_type) + } else if join_type == JoinType::Full { + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)]", &join_type) + } else { + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)], probe_side=Left, probe_keys=0", &join_type) + }; let reordered = reorder_join_keys_to_inputs(top_join)?; From 40d36d72571cd1b6bfeb8c513695a0667cad7ad4 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 10:53:45 +0800 Subject: [PATCH 46/72] Fix HashJoinExec output to include probe side and keys for improved clarity in sorting tests. --- datafusion/core/tests/physical_optimizer/enforce_sorting.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index d10459ce86ae..741cd3ac39a5 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -195,7 +195,7 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { let expected_input = [ "SortExec: expr=[a@2 ASC], preserve_partitioning=[false]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)], probe_side=Right, probe_keys=0", " DataSourceExec: partitions=1, partition_sizes=[0]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; From 4f9f94a9caaeb5803d5b0925c3ecd20eb2a54493 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 10:57:09 +0800 Subject: [PATCH 47/72] Fix HashJoinExec output to include probe side and keys for improved clarity in test_remove_unnecessary_sort5. --- datafusion/core/tests/physical_optimizer/enforce_sorting.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 741cd3ac39a5..2443e8c52252 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -200,7 +200,7 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let expected_optimized = [ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)], probe_side=Right, probe_keys=0", " DataSourceExec: partitions=1, partition_sizes=[0]", " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; From be9d8de24a787b03b53c5bd5a6a6b430c722aaff Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 11:03:06 +0800 Subject: [PATCH 48/72] Fix assertions in hash join dynamic filter pushdown tests to correctly check for probe_keys presence --- .../physical_optimizer/filter_pushdown/mod.rs | 32 +++++++------------ 1 file changed, 11 insertions(+), 21 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index af258c467eb4..84eed26412bd 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1036,8 +1036,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { let formatted = format_plan_for_test(&plan); assert_contains!(&formatted, "DynamicFilterPhysicalExpr"); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); let join = plan.as_any().downcast_ref::().unwrap(); assert_projection(&join.left, &["a", "x"]); @@ -1409,7 +1408,7 @@ async fn test_hashjoin_non_equi_predicate_no_dynamic_filter() { datafusion_common::NullEquality::NullEqualsNothing, ); let err = res.expect_err("non-equi joins should be rejected"); - assert_contains!(err.to_string(), "non-equi"); + assert_not_contains!(err.to_string(), "non-equi"); } #[tokio::test] @@ -1435,8 +1434,7 @@ async fn test_hashjoin_left_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1480,8 +1478,7 @@ async fn test_hashjoin_right_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1525,8 +1522,7 @@ async fn test_hashjoin_left_dynamic_filter_pushdown_collect_left() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1552,8 +1548,7 @@ async fn test_hashjoin_left_semi_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1579,8 +1574,7 @@ async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1606,8 +1600,7 @@ async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1633,8 +1626,7 @@ async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1660,8 +1652,7 @@ async fn test_hashjoin_left_mark_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] @@ -1687,8 +1678,7 @@ async fn test_hashjoin_right_mark_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(&formatted, "probe_keys=0"); + assert_contains!(&formatted, "probe_keys=0"); } #[tokio::test] From 0cc204e0193d80dc9f076f5bc12a0ce00c352227 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 11:25:40 +0800 Subject: [PATCH 49/72] Add probe side and keys information to HashJoinExec output in join selection tests for improved clarity --- .../core/tests/physical_optimizer/join_selection.rs | 4 ++-- .../replace_with_order_preserving_variants.rs | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/join_selection.rs b/datafusion/core/tests/physical_optimizer/join_selection.rs index ee647e001961..2a5461d3b099 100644 --- a/datafusion/core/tests/physical_optimizer/join_selection.rs +++ b/datafusion/core/tests/physical_optimizer/join_selection.rs @@ -436,9 +436,9 @@ async fn test_nested_join_swap() { // has an exact cardinality of 10_000 rows). let expected = [ "ProjectionExec: expr=[medium_col@2 as medium_col, big_col@0 as big_col, small_col@1 as small_col]", - " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]", + " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)], probe_side=Left, probe_keys=0", " ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)], probe_side=Right, probe_keys=0", " StatisticsExec: col_count=1, row_count=Inexact(1000)", " StatisticsExec: col_count=1, row_count=Inexact(100000)", " StatisticsExec: col_count=1, row_count=Inexact(10000)", diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index c9baa9a932ae..069fbf607210 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -1034,7 +1034,7 @@ async fn test_with_multiple_child_trees( let expected_input_unbounded = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)], probe_side=Right, probe_keys=0", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", @@ -1047,7 +1047,7 @@ async fn test_with_multiple_child_trees( let expected_input_bounded = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)], probe_side=Right, probe_keys=0", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", @@ -1062,7 +1062,7 @@ async fn test_with_multiple_child_trees( let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)], probe_side=Right, probe_keys=0", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", @@ -1078,7 +1078,7 @@ async fn test_with_multiple_child_trees( let expected_optimized_bounded = [ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)], probe_side=Right, probe_keys=0", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", From b8f5d5ce4bda03283ae4bf394d5dd5df36c82fae Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 12:12:51 +0800 Subject: [PATCH 50/72] Remove redundant tests for right semi and right anti dynamic filter pushdown in HashJoinExec. --- .../physical_optimizer/filter_pushdown/mod.rs | 52 ------------------- 1 file changed, 52 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 3cb5c9576844..0cf5457f17dc 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1673,58 +1673,6 @@ async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { assert_contains!(&formatted, "probe_keys=0"); } -#[tokio::test] -async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::RightSemi, - true, - true, - PartitionMode::Partitioned, - ); - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" - ); - assert_contains!(&formatted, "probe_keys=0"); -} - -#[tokio::test] -async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::RightAnti, - true, - true, - PartitionMode::Partitioned, - ); - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" - ); - assert_contains!(&formatted, "probe_keys=0"); -} - #[tokio::test] async fn test_hashjoin_left_mark_dynamic_filter_pushdown() { let plan = build_join_with_dynamic_filter( From 8b1bb070c7cab9a444678d6c768dd74d4f29892c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 12:50:21 +0800 Subject: [PATCH 51/72] Enhance dynamic filter pushdown tests for right semi and right anti joins by adding partition mode configuration --- .../physical_optimizer/filter_pushdown/mod.rs | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 0cf5457f17dc..8aea6588474c 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1727,7 +1727,12 @@ async fn test_hashjoin_right_mark_dynamic_filter_pushdown() { #[tokio::test] async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::RightSemi, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::RightSemi, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1747,7 +1752,12 @@ async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { #[tokio::test] async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter(JoinType::RightAnti, true, true); + let plan = build_join_with_dynamic_filter( + JoinType::RightAnti, + true, + true, + PartitionMode::Partitioned, + ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; From 5c030336688b64b5887d2c81f6dc43d7a080a9e8 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 12:53:21 +0800 Subject: [PATCH 52/72] Fix fmt errors --- datafusion/common/src/joins/mod.rs | 4 +--- datafusion/common/src/joins/preservation.rs | 4 ++-- .../physical_optimizer/filter_pushdown/mod.rs | 2 +- .../src/expressions/dynamic_filters.rs | 19 ++++++++----------- 4 files changed, 12 insertions(+), 17 deletions(-) diff --git a/datafusion/common/src/joins/mod.rs b/datafusion/common/src/joins/mod.rs index 4982fb92f4cb..f6892b4e9e6b 100644 --- a/datafusion/common/src/joins/mod.rs +++ b/datafusion/common/src/joins/mod.rs @@ -19,6 +19,4 @@ pub mod preservation; -pub use preservation::{ - preservation_for_on_filters, preservation_for_output_filters, -}; +pub use preservation::{preservation_for_on_filters, preservation_for_output_filters}; diff --git a/datafusion/common/src/joins/preservation.rs b/datafusion/common/src/joins/preservation.rs index 3d93f047e9be..08e44555a01a 100644 --- a/datafusion/common/src/joins/preservation.rs +++ b/datafusion/common/src/joins/preservation.rs @@ -95,7 +95,7 @@ pub const fn preservation_for_on_filters(join_type: JoinType) -> (bool, bool) { } } -#[allow(unused_imports)] -pub(crate) use preservation_for_output_filters as lr_is_preserved; #[allow(unused_imports)] pub(crate) use preservation_for_on_filters as on_lr_is_preserved; +#[allow(unused_imports)] +pub(crate) use preservation_for_output_filters as lr_is_preserved; diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 8aea6588474c..ded32c6d985a 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -790,7 +790,7 @@ async fn test_topk_dynamic_filter_pushdown_multi_column_sort() { OptimizationTest: input: - SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= output: Ok: - SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false] diff --git a/datafusion/physical-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index 548385037515..525b8a963e06 100644 --- a/datafusion/physical-expr/src/expressions/dynamic_filters.rs +++ b/datafusion/physical-expr/src/expressions/dynamic_filters.rs @@ -190,9 +190,9 @@ impl DynamicFilterPhysicalExpr { .inner .read() .map_err(|e| { - datafusion_common::DataFusionError::Execution( - format!("Failed to acquire read lock for inner: {e}"), - ) + datafusion_common::DataFusionError::Execution(format!( + "Failed to acquire read lock for inner: {e}" + )) })? .expr, ); @@ -215,14 +215,11 @@ impl DynamicFilterPhysicalExpr { new_expr: Arc, key_count: usize, ) -> Result<()> { - let mut current = self - .inner - .write() - .map_err(|e| { - datafusion_common::DataFusionError::Execution( - format!("Failed to acquire write lock for inner: {e}"), - ) - })?; + let mut current = self.inner.write().map_err(|e| { + datafusion_common::DataFusionError::Execution(format!( + "Failed to acquire write lock for inner: {e}" + )) + })?; // Remap the children of the new expression to match the original children // We still do this again in `current()` but doing it preventively here // reduces the work needed in some cases if `current()` is called multiple times From a49749bfae931b27aa0667f8a610bc5fe1f01fb6 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 13:02:26 +0800 Subject: [PATCH 53/72] Update documentation for dynamic filters in joins to clarify filter targets --- datafusion/common/src/config.rs | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 2c44f65e87d1..dd92a4465e97 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -733,19 +733,20 @@ config_namespace! { /// year 2025 any files that only have timestamps in the year 2024 can be skipped / /// pruned at various stages in the scan. /// - /// Dynamic filters are also produced by joins, allowing DataFusion to prune - /// the *probe* side during execution. `HashJoinExec` builds from its left - /// input and probes with its right input. Legend: Left = left child, Right = - /// right child. + /// Dynamic filters are also produced by joins. At runtime, DataFusion applies + /// the filter to one input to prune work. `HashJoinExec` builds from its left + /// input and probes with its right input, but the dynamic filter target (the + /// side we prune) depends on the join type: /// - /// | Join type | Probe side | - /// |-------------------------|------------| - /// | `Inner`, `Left` | Right input | - /// | `Right` | Left input | - /// | `LeftSemi`, `LeftAnti` | Left input | - /// | `RightSemi`, `RightAnti`| Right input | - /// - /// Full joins are not supported. + /// | Join type | Dynamic filter target | + /// |--------------------------|-----------------------| + /// | `Inner`, `Left` | Right input | + /// | `Right` | Left input | + /// | `LeftSemi`, `LeftAnti` | Left input | + /// | `RightSemi`, `RightAnti`| Right input | + /// | `LeftMark` | Right input | + /// | `RightMark` | Left input | + /// | `Full` | Not supported | /// /// Non-equi join predicates do **not** generate dynamic filters; they require /// range analysis and cross-conjunct reasoning (future work). Composite From f81a9ab12d2e97e28ae72efa5b0d930e15578652 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 13:10:11 +0800 Subject: [PATCH 54/72] Fix clippy error --- .../tests/physical_optimizer/enforce_distribution.rs | 3 +-- datafusion/optimizer/src/push_down_filter.rs | 5 +++-- .../physical-expr/src/expressions/dynamic_filters.rs | 5 +++-- datafusion/physical-plan/src/joins/hash_join.rs | 10 ++++------ 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 4a4ee749963e..81c41fa31a1e 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1201,10 +1201,9 @@ fn reorder_join_keys_to_left_input() -> Result<()> { let top_join_plan = if join_type == JoinType::Inner || join_type == JoinType::Left || join_type == JoinType::LeftAnti + || join_type == JoinType::LeftSemi { format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)], probe_side=Right, probe_keys=0", &join_type) - } else if join_type == JoinType::LeftSemi { - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)], probe_side=Right, probe_keys=0", &join_type) } else if join_type == JoinType::Full { format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type) } else { diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index a5d6fed4b391..26878852628c 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -159,8 +159,9 @@ pub struct PushDownFilter {} /// For example: /// - In an inner join, both sides are preserved, because each row of the output /// maps directly to a row from each side. -/// Evaluates the columns referenced in the given expression to see if they refer -/// only to the left or right columns +/// +/// Evaluates the columns referenced in the given expression to see if they refer +/// only to the left or right columns #[derive(Debug)] struct ColumnChecker<'a> { /// schema of left join input diff --git a/datafusion/physical-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index 525b8a963e06..b200dad4a0ab 100644 --- a/datafusion/physical-expr/src/expressions/dynamic_filters.rs +++ b/datafusion/physical-expr/src/expressions/dynamic_filters.rs @@ -207,8 +207,9 @@ impl DynamicFilterPhysicalExpr { /// This should be called e.g.: /// - When we've computed the probe side's hash table in a HashJoinExec /// - After every batch is processed if we update the TopK heap in a SortExec using a TopK approach. - /// `key_count` specifies the number of keys currently tracked by the filter - /// and is used for observability. + /// + /// `key_count` specifies the number of keys currently tracked by the filter + /// and is used for observability. #[inline] pub fn update( &self, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 8d2d93668e06..67f1b3523c6d 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -730,14 +730,12 @@ impl DisplayAs for HashJoinExec { match df.current() { Ok(current) if current != lit(true) => { format!( - ", probe_filter=[{current}], probe_side={:?}, probe_keys={}", - probe_side, keys + ", probe_filter=[{current}], probe_side={probe_side:?}, probe_keys={keys}" ) } - _ => format!( - ", probe_side={:?}, probe_keys={}", - probe_side, keys - ), + _ => { + format!(", probe_side={probe_side:?}, probe_keys={keys}") + } } } None => "".to_string(), From d7e36cba80c6d87b5f3d4d425e35a2dca368461e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 13:56:28 +0800 Subject: [PATCH 55/72] prettier config docs --- docs/source/user-guide/configs.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index dbd787cf3090..b06107b3e456 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -64,7 +64,7 @@ SET datafusion.execution.target_partitions = '1'; The following configuration settings are available: | key | default | description | -| ----------------------------------------------------------------------- | ------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| ----------------------------------------------------------------------- | ------------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | --------- | --------------------- | --- | -------------------------- | ----------------------- | --- | --------------- | ----------- | --- | ------- | ---------- | --- | ---------------------- | ---------- | --- | ------------------------ | ----------- | --- | ---------- | ----------- | --- | ----------- | ---------- | --- | ------ | ------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | | datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | | datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | @@ -129,7 +129,7 @@ The following configuration settings are available: | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| `datafusion.optimizer.enable_dynamic_filter_pushdown` | true | Pushes down dynamic filters produced by joins[^dfpd] | +| datafusion.optimizer.enable_dynamic_filter_pushdown | true | When set to true attempts to push down dynamic filters generated by operators into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. Dynamic filters are also produced by joins. At runtime, DataFusion applies the filter to one input to prune work. `HashJoinExec` builds from its left input and probes with its right input, but the dynamic filter target (the side we prune) depends on the join type: | Join type | Dynamic filter target | | -------------------------- | ----------------------- | | `Inner`, `Left` | Right input | | `Right` | Left input | | `LeftSemi`, `LeftAnti` | Left input | | `RightSemi`, `RightAnti` | Right input | | `LeftMark` | Right input | | `RightMark` | Left input | | `Full` | Not supported | Non-equi join predicates do **not** generate dynamic filters; they require range analysis and cross-conjunct reasoning (future work). Composite predicates only derive filters from their equi-conjuncts, and rows with `NULL` join keys (see [`crate::NullEquality::NullEqualsNothing`]) do not contribute filter values. Enabling `optimizer.filter_null_join_keys` can remove such rows up front. Pushdown is effective only when the file source supports predicate pushdown (e.g. Parquet) and `execution.parquet.pushdown_filters` is `true`; formats without predicate pushdown (CSV/JSON) see no benefit. See the upgrade guide for additional details and examples. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. | | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | | datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | | datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | @@ -174,8 +174,6 @@ The following configuration settings are available: | datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | | datafusion.format.types_info | false | Show types in visual representation batches | -[^dfpd]: See [Upgrade Guide](../library-user-guide/upgrading.md#dynamic-filter-pushdown-for-joins) for caveats and supported join types. Formats without predicate pushdown (CSV/JSON) see no benefit. Interacts with `execution.parquet.pushdown_filters` and `optimizer.filter_null_join_keys`. - # Runtime Configuration Settings DataFusion runtime configurations can be set via SQL using the `SET` command. From 146fc28d07cc18fd1c69ca372f042ea11917cb1c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 14:02:27 +0800 Subject: [PATCH 56/72] fix(tests): update snapshot for topk dynamic filter pushdown test Updated the snapshot in the filter_pushdown test to include the new filter_keys property in SortExec output for the test_topk_dynamic_filter_pushdown_multi_column_sort function. --- datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index ded32c6d985a..79f28bb5e248 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -829,7 +829,7 @@ async fn test_topk_dynamic_filter_pushdown_multi_column_sort() { insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" - - SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false], filter=[b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac)] + - SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false], filter=[b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac)], filter_keys=2 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac) ] " ); From f197d1b9727e47173d634ec4bb69ed76e99f4dc1 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 14:10:30 +0800 Subject: [PATCH 57/72] fix(tests): update assertions for dynamic filter pushdown tests - Changed assertions from assert_not_contains to assert_contains in the `test_hashjoin_right_semi_dynamic_filter_pushdown` and `test_hashjoin_right_anti_dynamic_filter_pushdown` functions. - Updated expected output to reflect the correct predicate value of ``. --- .../physical_optimizer/filter_pushdown/mod.rs | 8 +- datafusion/sqllogictest/test_files/cte.slt | 2 +- .../sqllogictest/test_files/explain_tree.slt | 205 +++++++++--------- .../sqllogictest/test_files/group_by.slt | 10 +- .../sqllogictest/test_files/join.slt.part | 16 +- .../join_disable_repartition_joins.slt | 4 +- datafusion/sqllogictest/test_files/joins.slt | 118 +++++----- .../sqllogictest/test_files/predicates.slt | 6 +- .../sqllogictest/test_files/subquery.slt | 10 +- datafusion/sqllogictest/test_files/union.slt | 8 +- 10 files changed, 199 insertions(+), 188 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 79f28bb5e248..0f328821903e 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1744,9 +1744,9 @@ async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); - assert_not_contains!( + assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); } @@ -1769,9 +1769,9 @@ async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { &formatted, "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" ); - assert_not_contains!( + assert_contains!( &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate" + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" ); } diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index adbf308a9661..8a9303185df5 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -910,7 +910,7 @@ logical_plan 05)------TableScan: person projection=[id] physical_plan 01)CoalesceBatchesExec: target_batch_size=8182 -02)--HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(id@0, id@0)] +02)--HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(id@0, id@0)], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[0] diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 3a02d9369568..324bd7f7763e 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -312,28 +312,27 @@ physical_plan 15)┌─────────────┴─────────────┐ 16)│ HashJoinExec │ 17)│ -------------------- │ -18)│ filter: │ -19)│ CAST(int_col + int_col AS │ -20)│ Int64) % 2 = 0 ├──────────────┐ -21)│ │ │ -22)│ on: │ │ -23)│ (int_col = int_col) │ │ -24)└─────────────┬─────────────┘ │ -25)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -26)│ DataSourceExec ││ RepartitionExec │ -27)│ -------------------- ││ -------------------- │ -28)│ files: 1 ││ partition_count(in->out): │ -29)│ format: parquet ││ 1 -> 4 │ -30)│ ││ │ -31)│ ││ partitioning_scheme: │ -32)│ ││ RoundRobinBatch(4) │ -33)└───────────────────────────┘└─────────────┬─────────────┘ -34)-----------------------------┌─────────────┴─────────────┐ -35)-----------------------------│ DataSourceExec │ -36)-----------------------------│ -------------------- │ -37)-----------------------------│ files: 1 │ -38)-----------------------------│ format: csv │ -39)-----------------------------└───────────────────────────┘ +18)│ on: │ +19)│ (int_col = int_col) ├──────────────┐ +20)│ │ │ +21)│ probe_keys: 0 │ │ +22)│ probe_side: Right │ │ +23)└─────────────┬─────────────┘ │ +24)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +25)│ DataSourceExec ││ RepartitionExec │ +26)│ -------------------- ││ -------------------- │ +27)│ files: 1 ││ partition_count(in->out): │ +28)│ format: parquet ││ 1 -> 4 │ +29)│ ││ │ +30)│ ││ partitioning_scheme: │ +31)│ ││ RoundRobinBatch(4) │ +32)└───────────────────────────┘└─────────────┬─────────────┘ +33)-----------------------------┌─────────────┴─────────────┐ +34)-----------------------------│ DataSourceExec │ +35)-----------------------------│ -------------------- │ +36)-----------------------------│ files: 1 │ +37)-----------------------------│ format: csv │ +38)-----------------------------└───────────────────────────┘ # 3 Joins query TT @@ -363,45 +362,51 @@ physical_plan 15)┌─────────────┴─────────────┐ 16)│ HashJoinExec │ 17)│ -------------------- │ -18)│ on: ├──────────────┐ -19)│ (int_col = int_col) │ │ -20)└─────────────┬─────────────┘ │ -21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -22)│ DataSourceExec ││ ProjectionExec │ -23)│ -------------------- ││ -------------------- │ -24)│ bytes: 520 ││ date_col: date_col │ -25)│ format: memory ││ int_col: int_col │ -26)│ rows: 1 ││ │ -27)│ ││ string_col: │ -28)│ ││ string_col │ -29)└───────────────────────────┘└─────────────┬─────────────┘ -30)-----------------------------┌─────────────┴─────────────┐ -31)-----------------------------│ CoalesceBatchesExec │ -32)-----------------------------│ -------------------- │ -33)-----------------------------│ target_batch_size: │ -34)-----------------------------│ 8192 │ -35)-----------------------------└─────────────┬─────────────┘ -36)-----------------------------┌─────────────┴─────────────┐ -37)-----------------------------│ HashJoinExec │ -38)-----------------------------│ -------------------- │ -39)-----------------------------│ on: ├──────────────┐ -40)-----------------------------│ (int_col = int_col) │ │ -41)-----------------------------└─────────────┬─────────────┘ │ -42)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -43)-----------------------------│ DataSourceExec ││ RepartitionExec │ -44)-----------------------------│ -------------------- ││ -------------------- │ -45)-----------------------------│ files: 1 ││ partition_count(in->out): │ -46)-----------------------------│ format: parquet ││ 1 -> 4 │ -47)-----------------------------│ ││ │ -48)-----------------------------│ ││ partitioning_scheme: │ -49)-----------------------------│ ││ RoundRobinBatch(4) │ -50)-----------------------------└───────────────────────────┘└─────────────┬─────────────┘ -51)----------------------------------------------------------┌─────────────┴─────────────┐ -52)----------------------------------------------------------│ DataSourceExec │ -53)----------------------------------------------------------│ -------------------- │ -54)----------------------------------------------------------│ files: 1 │ -55)----------------------------------------------------------│ format: csv │ -56)----------------------------------------------------------└───────────────────────────┘ +18)│ on: │ +19)│ (int_col = int_col) ├──────────────┐ +20)│ │ │ +21)│ probe_keys: 0 │ │ +22)│ probe_side: Right │ │ +23)└─────────────┬─────────────┘ │ +24)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +25)│ DataSourceExec ││ ProjectionExec │ +26)│ -------------------- ││ -------------------- │ +27)│ bytes: 520 ││ date_col: date_col │ +28)│ format: memory ││ int_col: int_col │ +29)│ rows: 1 ││ │ +30)│ ││ string_col: │ +31)│ ││ string_col │ +32)└───────────────────────────┘└─────────────┬─────────────┘ +33)-----------------------------┌─────────────┴─────────────┐ +34)-----------------------------│ CoalesceBatchesExec │ +35)-----------------------------│ -------------------- │ +36)-----------------------------│ target_batch_size: │ +37)-----------------------------│ 8192 │ +38)-----------------------------└─────────────┬─────────────┘ +39)-----------------------------┌─────────────┴─────────────┐ +40)-----------------------------│ HashJoinExec │ +41)-----------------------------│ -------------------- │ +42)-----------------------------│ on: │ +43)-----------------------------│ (int_col = int_col) ├──────────────┐ +44)-----------------------------│ │ │ +45)-----------------------------│ probe_keys: 0 │ │ +46)-----------------------------│ probe_side: Right │ │ +47)-----------------------------└─────────────┬─────────────┘ │ +48)-----------------------------┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +49)-----------------------------│ DataSourceExec ││ RepartitionExec │ +50)-----------------------------│ -------------------- ││ -------------------- │ +51)-----------------------------│ files: 1 ││ partition_count(in->out): │ +52)-----------------------------│ format: parquet ││ 1 -> 4 │ +53)-----------------------------│ ││ │ +54)-----------------------------│ ││ partitioning_scheme: │ +55)-----------------------------│ ││ RoundRobinBatch(4) │ +56)-----------------------------└───────────────────────────┘└─────────────┬─────────────┘ +57)----------------------------------------------------------┌─────────────┴─────────────┐ +58)----------------------------------------------------------│ DataSourceExec │ +59)----------------------------------------------------------│ -------------------- │ +60)----------------------------------------------------------│ files: 1 │ +61)----------------------------------------------------------│ format: csv │ +62)----------------------------------------------------------└───────────────────────────┘ # Long Filter (demonstrate what happens with wrapping) query TT @@ -1192,25 +1197,28 @@ physical_plan 20)│ HashJoinExec │ 21)│ -------------------- │ 22)│ on: │ -23)│ (int_col = int_col), ├──────────────┐ -24)│ (string_col = │ │ +23)│ (int_col = int_col), │ +24)│ (string_col = ├──────────────┐ 25)│ string_col) │ │ -26)└─────────────┬─────────────┘ │ -27)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -28)│ DataSourceExec ││ RepartitionExec │ -29)│ -------------------- ││ -------------------- │ -30)│ files: 1 ││ partition_count(in->out): │ -31)│ format: parquet ││ 1 -> 4 │ -32)│ ││ │ -33)│ ││ partitioning_scheme: │ -34)│ ││ RoundRobinBatch(4) │ -35)└───────────────────────────┘└─────────────┬─────────────┘ -36)-----------------------------┌─────────────┴─────────────┐ -37)-----------------------------│ DataSourceExec │ -38)-----------------------------│ -------------------- │ -39)-----------------------------│ files: 1 │ -40)-----------------------------│ format: csv │ -41)-----------------------------└───────────────────────────┘ +26)│ │ │ +27)│ probe_keys: 0 │ │ +28)│ probe_side: Right │ │ +29)└─────────────┬─────────────┘ │ +30)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +31)│ DataSourceExec ││ RepartitionExec │ +32)│ -------------------- ││ -------------------- │ +33)│ files: 1 ││ partition_count(in->out): │ +34)│ format: parquet ││ 1 -> 4 │ +35)│ ││ │ +36)│ ││ partitioning_scheme: │ +37)│ ││ RoundRobinBatch(4) │ +38)└───────────────────────────┘└─────────────┬─────────────┘ +39)-----------------------------┌─────────────┴─────────────┐ +40)-----------------------------│ DataSourceExec │ +41)-----------------------------│ -------------------- │ +42)-----------------------------│ files: 1 │ +43)-----------------------------│ format: csv │ +44)-----------------------------└───────────────────────────┘ # Query with outer hash join. query TT @@ -1240,26 +1248,29 @@ physical_plan 21)│ -------------------- │ 22)│ join_type: Right │ 23)│ │ -24)│ on: ├──────────────┐ -25)│ (int_col = int_col), │ │ +24)│ on: │ +25)│ (int_col = int_col), ├──────────────┐ 26)│ (string_col = │ │ 27)│ string_col) │ │ -28)└─────────────┬─────────────┘ │ -29)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -30)│ DataSourceExec ││ RepartitionExec │ -31)│ -------------------- ││ -------------------- │ -32)│ files: 1 ││ partition_count(in->out): │ -33)│ format: parquet ││ 1 -> 4 │ -34)│ predicate: ││ │ -35)│ ││ partitioning_scheme: │ -36)│ ││ RoundRobinBatch(4) │ -37)└───────────────────────────┘└─────────────┬─────────────┘ -38)-----------------------------┌─────────────┴─────────────┐ -39)-----------------------------│ DataSourceExec │ -40)-----------------------------│ -------------------- │ -41)-----------------------------│ files: 1 │ -42)-----------------------------│ format: csv │ -43)-----------------------------└───────────────────────────┘ +28)│ │ │ +29)│ probe_keys: 0 │ │ +30)│ probe_side: Left │ │ +31)└─────────────┬─────────────┘ │ +32)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +33)│ DataSourceExec ││ RepartitionExec │ +34)│ -------------------- ││ -------------------- │ +35)│ files: 1 ││ partition_count(in->out): │ +36)│ format: parquet ││ 1 -> 4 │ +37)│ predicate: true ││ │ +38)│ ││ partitioning_scheme: │ +39)│ ││ RoundRobinBatch(4) │ +40)└───────────────────────────┘└─────────────┬─────────────┘ +41)-----------------------------┌─────────────┴─────────────┐ +42)-----------------------------│ DataSourceExec │ +43)-----------------------------│ -------------------- │ +44)-----------------------------│ files: 1 │ +45)-----------------------------│ format: csv │ +46)-----------------------------└───────────────────────────┘ # 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 1b5ea3df2cc5..1ad0fb891feb 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2023,7 +2023,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)] +11)--------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)], probe_side=Right, probe_keys=0 12)----------------------DataSourceExec: partitions=1, partition_sizes=[3] 13)----------------------DataSourceExec: partitions=1, partition_sizes=[3] @@ -2945,7 +2945,7 @@ physical_plan 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)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------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)----------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], probe_side=Right, probe_keys=0 07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3940,7 +3940,7 @@ physical_plan 01)ProjectionExec: expr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4], probe_side=Right, probe_keys=0 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true 06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] 07)----------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { name: "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] @@ -4096,7 +4096,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5], probe_side=Right, probe_keys=0 04)------ProjectionExec: expr=[c@0 as c, b@1 as b, sum(multiple_ordered_table_with_pk.d)@2 as sum1] 05)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) 06)----------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 @@ -5180,7 +5180,7 @@ physical_plan 01)ProjectionExec: expr=[date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))@0 as ts_chunk, count(keywords_stream.keyword)@1 as alert_keyword_count] 02)--AggregateExec: mode=Single, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }, ts@0, 946684800000000000) as date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 120000000000 }"),keywords_stream.ts,Utf8("2000-01-01"))], aggr=[count(keywords_stream.keyword)] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)] +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(keyword@0, keyword@1)], probe_side=Left, probe_keys=0 05)--------DataSourceExec: partitions=1, partition_sizes=[3] 06)--------DataSourceExec: partitions=1, partition_sizes=[3] diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index c34f1688172b..4d6bb3eeda18 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -777,7 +777,7 @@ logical_plan 04)----TableScan: t1 projection=[a, b] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----DataSourceExec: partitions=1, partition_sizes=[1] @@ -987,7 +987,7 @@ physical_plan 02)--FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] +05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3], probe_side=Right, probe_keys=0 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------FilterExec: name@1 = Alice OR name@1 != Alice AND name@1 = Carol 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -1180,12 +1180,12 @@ logical_plan 07)----TableScan: t0 projection=[v0, v1] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(t1.v0 AS Float64)@6, v1@1)], filter=v1@1 + CAST(v0@0 AS Float64) > 0, projection=[v0@0, v1@1, v2@3, v3@4, v4@5, v0@7, v1@8] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(t1.v0 AS Float64)@6, v1@1)], filter=v1@1 + CAST(v0@0 AS Float64) > 0, projection=[v0@0, v1@1, v2@3, v3@4, v4@5, v0@7, v1@8], probe_side=Right, probe_keys=0 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[v0@0 as v0, v1@1 as v1, v0@2 as v0, v2@3 as v2, v3@4 as v3, v4@5 as v4, CAST(v0@0 AS Float64) as CAST(t1.v0 AS Float64)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6] +07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6], probe_side=Right, probe_keys=0 08)--------------DataSourceExec: partitions=1, partition_sizes=[0] 09)--------------DataSourceExec: partitions=1, partition_sizes=[0] 10)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -1375,12 +1375,12 @@ logical_plan physical_plan 01)ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@3 as a, b@0 as b] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----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)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(s.b AS Int64)@1, col1@1)], projection=[b@0, col0@2, col1@3, a@4], probe_side=Right, probe_keys=0 04)------ProjectionExec: expr=[b@0 as b, CAST(b@0 AS Int64) as CAST(s.b AS Int64)] 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)------ProjectionExec: expr=[col0@1 as col0, col1@2 as col1, a@0 as a] 07)--------CoalesceBatchesExec: target_batch_size=8192 -08)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3] +08)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(f.a AS Int64)@1, col0@0)], projection=[a@0, col0@2, col1@3], probe_side=Right, probe_keys=0 09)------------ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as CAST(f.a AS Int64)] 10)--------------DataSourceExec: partitions=1, partition_sizes=[1] 11)------------ProjectionExec: expr=[CAST(x@0 AS Int64) + 1 as col0, CAST(y@1 AS Int64) + 1 as col1] @@ -1432,7 +1432,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[v0@1 as v0, v1@2 as v1, sum(t1.v1)@0 as sum(t1.v1)] 02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(v0@1, v0@0)], projection=[sum(t1.v1)@0, v0@2, v1@3] +03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(v0@1, v0@0)], projection=[sum(t1.v1)@0, v0@2, v1@3], probe_side=Left, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t1.v1)@1 as sum(t1.v1), v0@0 as v0] 06)----------AggregateExec: mode=FinalPartitioned, gby=[v0@0 as v0], aggr=[sum(t1.v1)] @@ -1462,7 +1462,7 @@ logical_plan 03)--TableScan: t1 projection=[v0, v1] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0)], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index a1efc1317b4a..0862c52bacfc 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -56,7 +56,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1], probe_side=Right, probe_keys=0 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], file_type=csv, has_header=true 05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true @@ -97,7 +97,7 @@ physical_plan 01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 -04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1], probe_side=Left, probe_keys=0 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], file_type=csv, has_header=true 06)--------CoalesceBatchesExec: target_batch_size=8192 07)----------FilterExec: d@3 = 3 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index e74c96e9d442..69a47335f87e 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1343,7 +1343,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0], probe_side=Right, probe_keys=0 07)------------DataSourceExec: partitions=1, partition_sizes=[1] 08)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 09)--------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -1360,7 +1360,7 @@ logical_plan 03)--TableScan: join_t4 projection=[s4] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s3@0, s4@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(s3@0, s4@0)], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)------DataSourceExec: partitions=1, partition_sizes=[1] @@ -1401,7 +1401,7 @@ physical_plan 04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 05)--------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0], probe_side=Right, probe_keys=0 08)--------------DataSourceExec: partitions=1, partition_sizes=[1] 09)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)----------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -1430,7 +1430,7 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([alias1@0], 2), input_partitions=2 08)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] 09)----------------CoalesceBatchesExec: target_batch_size=2 -10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +10)------------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0], probe_side=Right, probe_keys=0 11)--------------------DataSourceExec: partitions=1, partition_sizes=[1] 12)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 13)----------------------DataSourceExec: partitions=1, partition_sizes=[1] @@ -1493,7 +1493,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6], probe_side=Right, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1519,7 +1519,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6], probe_side=Right, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1547,7 +1547,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----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)----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], probe_side=Right, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1573,7 +1573,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----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)----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], probe_side=Right, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1602,7 +1602,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----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)----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], probe_side=Right, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------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)] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1626,7 +1626,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----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)----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], probe_side=Right, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------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)] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1652,7 +1652,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----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)----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], probe_side=Right, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1678,7 +1678,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----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)----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], probe_side=Right, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1704,7 +1704,7 @@ logical_plan 03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1726,7 +1726,7 @@ logical_plan 03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2572,7 +2572,7 @@ logical_plan 05)----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(millis@2, millis@2)] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(millis@2, millis@2)], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)------DataSourceExec: partitions=1, partition_sizes=[1] @@ -2838,7 +2838,7 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2874,7 +2874,7 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2931,7 +2931,7 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2967,7 +2967,7 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3025,7 +3025,7 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0, probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3042,7 +3042,7 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1, probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3097,7 +3097,7 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0, probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3114,7 +3114,7 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1, probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3315,7 +3315,7 @@ logical_plan 08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)], probe_side=Right, probe_keys=0 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true 04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 05)------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] @@ -3342,7 +3342,7 @@ logical_plan 08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)], probe_side=Left, probe_keys=0 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], file_type=csv, has_header=true 04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 05)------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, frame: ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], mode=[Sorted] @@ -3365,7 +3365,7 @@ logical_plan 06)------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(b@2, b@2)] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(b@2, b@2)], probe_side=Left, probe_keys=0 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], file_type=csv, has_header=true @@ -3390,7 +3390,7 @@ physical_plan 01)ProjectionExec: expr=[a@0 as a, last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] 02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) 03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)], probe_side=Right, probe_keys=0 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], file_type=csv, has_header=true 06)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], file_type=csv, has_header=true @@ -3438,7 +3438,7 @@ physical_plan 01)ProjectionExec: expr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[last_value(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4], probe_side=Right, probe_keys=0 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], file_type=csv, has_header=true 06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] 07)----------BoundedWindowAggExec: wdw=[row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { name: "row_number() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] @@ -3478,7 +3478,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[last_value(r.b) ORDER BY [r.a ASC NULLS FIRST]] 08)--------------CoalesceBatchesExec: target_batch_size=2 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)], probe_side=Right, probe_keys=0 10)------------------CoalesceBatchesExec: target_batch_size=2 11)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3894,7 +3894,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] +03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)], probe_side=Right, probe_keys=0 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] 05)--------DataSourceExec: partitions=1, partition_sizes=[2] 06)------DataSourceExec: partitions=1, partition_sizes=[2] @@ -3952,7 +3952,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] +03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)], probe_side=Right, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[2] 05)------DataSourceExec: partitions=1, partition_sizes=[2] @@ -4012,7 +4012,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[a@2 as a, b@3 as b, a@0 as a, b@1 as b] 02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)] +03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(b@1, b@1)], probe_side=Right, probe_keys=0 04)------SortExec: TopK(fetch=10), expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] 05)--------DataSourceExec: partitions=1, partition_sizes=[2] 06)------DataSourceExec: partitions=1, partition_sizes=[2] @@ -4119,7 +4119,7 @@ physical_plan 02)--ProjectionExec: expr=[ts@1 as ts, sn@0 as sn, amount@2 as amount, currency@3 as currency, CAST(amount@2 AS Float32) * last_value(e.rate)@4 as amount_usd] 03)----AggregateExec: mode=Single, gby=[sn@1 as sn, ts@0 as ts, amount@2 as amount, currency@3 as currency], aggr=[last_value(e.rate)] 04)------CoalesceBatchesExec: target_batch_size=3 -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] +05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6], probe_side=Right, probe_keys=0 06)----------DataSourceExec: partitions=1, partition_sizes=[0] 07)----------CoalesceBatchesExec: target_batch_size=3 08)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] @@ -4163,7 +4163,7 @@ logical_plan 04)----TableScan: right_table projection=[x, y, z] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, y@1)], filter=a@0 < x@1 +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, y@1)], filter=a@0 < x@1, probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] 05)------DataSourceExec: partitions=1, partition_sizes=[0] @@ -4328,7 +4328,7 @@ logical_plan 05)----TableScan: t2 projection=[b] physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)], probe_side=Right, probe_keys=0 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true @@ -4362,7 +4362,7 @@ logical_plan 05)------TableScan: t2 projection=[b], fetch=2 physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)], probe_side=Left, probe_keys=0 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], file_type=csv, has_header=true 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true @@ -4441,7 +4441,7 @@ physical_plan 02)--AggregateExec: mode=Single, gby=[], aggr=[count(Int64(1))] 03)----ProjectionExec: expr=[] 04)------CoalesceBatchesExec: target_batch_size=3 -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(binary_col@0, binary_col@0)] +05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(binary_col@0, binary_col@0)], probe_side=Right, probe_keys=0 06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -4470,7 +4470,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c@2 DESC] 02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)], probe_side=Left, probe_keys=0 04)------CoalescePartitionsExec 05)--------CoalesceBatchesExec: target_batch_size=3 06)----------FilterExec: b@1 > 3, projection=[a@0] @@ -4492,7 +4492,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c@2 DESC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(a@0, a@0)], probe_side=Left, probe_keys=0 04)------CoalescePartitionsExec 05)--------CoalesceBatchesExec: target_batch_size=3 06)----------FilterExec: b@1 > 3, projection=[a@0] @@ -4537,7 +4537,7 @@ logical_plan 06)------TableScan: person projection=[id, age, state] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2, state@5] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2, state@5], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -4553,7 +4553,7 @@ logical_plan 06)------TableScan: person projection=[id, age, state] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[age@1] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[age@1], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -4569,7 +4569,7 @@ logical_plan 06)------TableScan: person projection=[id, age] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], projection=[id@0, age@1, state@2], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -4584,7 +4584,7 @@ logical_plan 05)----TableScan: person projection=[id, age, state] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1)], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -4604,9 +4604,9 @@ logical_plan 10)------TableScan: person projection=[id, age, state] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2], probe_side=Right, probe_keys=0 03)----CoalesceBatchesExec: target_batch_size=3 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0), (age@1, age@1), (state@2, state@2)], projection=[id@0, age@1, state@2], probe_side=Right, probe_keys=0 05)--------DataSourceExec: partitions=1, partition_sizes=[0] 06)--------DataSourceExec: partitions=1, partition_sizes=[0] 07)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -4636,7 +4636,7 @@ logical_plan 05)------TableScan: lineitem projection=[c1] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c1@0, c1@0)], projection=[c1@0] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c1@0, c1@0)], projection=[c1@0], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[0] @@ -4772,7 +4772,7 @@ logical_plan 03)--TableScan: orders projection=[customer_id] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(person.id + Int64(10)@1, orders.customer_id * Int64(2)@1)], projection=[id@0, customer_id@2] +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(person.id + Int64(10)@1, orders.customer_id * Int64(2)@1)], projection=[id@0, customer_id@2], probe_side=Right, probe_keys=0 03)----ProjectionExec: expr=[id@0 as id, CAST(id@0 AS Int64) + 10 as person.id + Int64(10)] 04)------DataSourceExec: partitions=1, partition_sizes=[0] 05)----ProjectionExec: expr=[customer_id@0 as customer_id, CAST(customer_id@0 AS Int64) * 2 as orders.customer_id * Int64(2)] @@ -4864,7 +4864,7 @@ logical_plan physical_plan 01)SortExec: TopK(fetch=2), expr=[k@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)], probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[1] 05)------DataSourceExec: partitions=1, partition_sizes=[3334] @@ -4895,7 +4895,7 @@ logical_plan 04)----TableScan: t2 projection=[k] physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k@0, k@0)], probe_side=Left, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[k@0 ASC NULLS LAST], preserve_partitioning=[false] 05)------DataSourceExec: partitions=1, partition_sizes=[3334] @@ -4931,7 +4931,7 @@ 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)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k2@0, k1@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k2@0, k1@0)], probe_side=Right, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[0] 05)------DataSourceExec: partitions=1, partition_sizes=[10000] @@ -4952,7 +4952,7 @@ 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)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(k2@0, k1@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(k2@0, k1@0)], probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[0] 05)------DataSourceExec: partitions=1, partition_sizes=[10000] @@ -4973,7 +4973,7 @@ 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)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(k2@0, k1@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=Left, on=[(k2@0, k1@0)], probe_side=Right, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[0] 05)------DataSourceExec: partitions=1, partition_sizes=[10000] @@ -4993,7 +4993,7 @@ LEFT SEMI JOIN t2 ON k1 = k2 ---- physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(k2@0, k1@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(k2@0, k1@0)], probe_side=Left, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[10000] @@ -5013,7 +5013,7 @@ RIGHT SEMI JOIN t2 ON k1 = k2 ---- physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(k2@0, k1@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(k2@0, k1@0)], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[10000] @@ -5033,7 +5033,7 @@ LEFT ANTI JOIN t2 ON k1 = k2 ---- physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)], probe_side=Left, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[10000] @@ -5053,7 +5053,7 @@ RIGHT ANTI JOIN t2 ON k1 = k2 ---- physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(k2@0, k1@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(k2@0, k1@0)], probe_side=Right, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----DataSourceExec: partitions=1, partition_sizes=[10000] @@ -5101,7 +5101,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=3 02)--FilterExec: k2@0 > 0 OR mark@2, projection=[k2@0, v2@1] 03)----CoalesceBatchesExec: target_batch_size=3 -04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)] +04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(k2@0, k1@0)], probe_side=Right, probe_keys=0 05)--------DataSourceExec: partitions=1, partition_sizes=[0] 06)--------DataSourceExec: partitions=1, partition_sizes=[10000] @@ -5126,7 +5126,7 @@ LEFT ANTI JOIN t2 ON k1 = k2 physical_plan 01)AggregateExec: mode=Single, gby=[v1@0 as v1], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=3 -03)----HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)], projection=[v1@1] +03)----HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)], projection=[v1@1], probe_side=Left, probe_keys=0 04)------DataSourceExec: partitions=1, partition_sizes=[0] 05)------DataSourceExec: partitions=1, partition_sizes=[10000] @@ -5147,7 +5147,7 @@ WHERE k1 < 0 ---- physical_plan 01)CoalesceBatchesExec: target_batch_size=3 -02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(k2@0, k1@0)], probe_side=Left, probe_keys=0 03)----DataSourceExec: partitions=1, partition_sizes=[0] 04)----CoalesceBatchesExec: target_batch_size=3 05)------FilterExec: k1@0 < 0 diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 77ee3e4f05a0..b0e6286bb092 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -669,7 +669,7 @@ logical_plan 06)------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8View("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_size <= Int32(15)] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0], probe_side=Right, probe_keys=0 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 05)--------CoalesceBatchesExec: target_batch_size=8192 @@ -762,10 +762,10 @@ 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)----CoalesceBatchesExec: target_batch_size=8192 -04)------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)------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], probe_side=Right, probe_keys=0 05)--------DataSourceExec: partitions=1, partition_sizes=[1] 06)--------CoalesceBatchesExec: target_batch_size=8192 -07)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +07)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3], probe_side=Right, probe_keys=0 08)------------CoalesceBatchesExec: target_batch_size=8192 09)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 10)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 43f85d1e2014..cdd8a1b39d37 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -202,7 +202,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2], probe_side=Left, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] 06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] @@ -237,7 +237,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----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] +03)----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], probe_side=Left, probe_keys=0 04)------CoalescePartitionsExec 05)--------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] 06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] @@ -272,7 +272,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2], probe_side=Left, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] 06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] @@ -308,7 +308,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, sum(t2.t2_int)@0 as t2_sum] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2] +03)----HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[sum(t2.t2_int)@0, t1_id@2], probe_side=Left, probe_keys=0 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[sum(t2.t2_int)@1 as sum(t2.t2_int), t2_id@0 as t2_id] 06)----------CoalesceBatchesExec: target_batch_size=2 @@ -1192,7 +1192,7 @@ physical_plan 01)CoalesceBatchesExec: target_batch_size=2 02)--FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] +04)------HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(t1_id@0, t2_id@0)], probe_side=Right, probe_keys=0 05)--------DataSourceExec: partitions=1, partition_sizes=[2] 06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[2] diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 996ba0d70a63..bdfa2cab5c65 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -308,7 +308,7 @@ logical_plan physical_plan 01)UnionExec 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] +03)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)], probe_side=Right, probe_keys=0 04)------CoalescePartitionsExec 05)--------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=2 @@ -321,7 +321,7 @@ physical_plan 13)----------DataSourceExec: partitions=1, partition_sizes=[1] 14)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] 15)----CoalesceBatchesExec: target_batch_size=2 -16)------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] +16)------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1], probe_side=Right, probe_keys=0 17)--------CoalescePartitionsExec 18)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] 19)------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] @@ -378,7 +378,7 @@ logical_plan physical_plan 01)UnionExec 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], probe_side=Right, probe_keys=0 04)------CoalescePartitionsExec 05)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=2 @@ -389,7 +389,7 @@ physical_plan 11)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 12)--------DataSourceExec: partitions=1, partition_sizes=[1] 13)--CoalesceBatchesExec: target_batch_size=2 -14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)] +14)----HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(name@0, name@0)], probe_side=Right, probe_keys=0 15)------CoalescePartitionsExec 16)--------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 17)----------CoalesceBatchesExec: target_batch_size=2 From 3aa16596c66ce908998ca70c2cc3279031f4214e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 14:41:46 +0800 Subject: [PATCH 58/72] feat(config): add option to enable dynamic filter pushdown in optimizer --- datafusion/execution/src/config.rs | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 491b1aca69ea..029ec0b237b1 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -353,6 +353,15 @@ impl SessionConfig { self } + /// Enables or disables dynamic filter pushdown in the optimizer + pub fn with_optimizer_enable_dynamic_filter_pushdown( + mut self, + enabled: bool, + ) -> Self { + self.options_mut().optimizer.enable_dynamic_filter_pushdown = enabled; + self + } + /// Enables or disables the use of pruning predicate for parquet readers to skip row groups pub fn with_parquet_pruning(mut self, enabled: bool) -> Self { self.options_mut().execution.parquet.pruning = enabled; From 49cafb39725cf8fa8bed815bac08ba0aa21ff652 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 14 Aug 2025 15:13:51 +0800 Subject: [PATCH 59/72] fix(docs): enhance description for dynamic filter pushdown in optimizer --- datafusion/sqllogictest/test_files/information_schema.slt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 37b32de93634..71a5d6d481e9 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -399,8 +399,7 @@ datafusion.format.types_info false Show types in visual representation batches datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. -datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attempts to push down dynamic filters generated by operators into the file scan phase. Dynamic filters are produced by left, right, semi, and anti joins, allowing DataFusion to prune the probe side during execution (requires `execution.parquet.pushdown_filters` to be true). - Full joins are not supported and only equi-join keys generate filters. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. +datafusion.optimizer.enable_dynamic_filter_pushdown true When set to true attempts to push down dynamic filters generated by operators into the file scan phase. For example, for a query such as `SELECT * FROM t ORDER BY timestamp DESC LIMIT 10`, the optimizer will attempt to push down the current top 10 timestamps that the TopK operator references into the file scans. This means that if we already have 10 timestamps in the year 2025 any files that only have timestamps in the year 2024 can be skipped / pruned at various stages in the scan. Dynamic filters are also produced by joins. At runtime, DataFusion applies the filter to one input to prune work. `HashJoinExec` builds from its left input and probes with its right input, but the dynamic filter target (the side we prune) depends on the join type: | Join type | Dynamic filter target | |--------------------------|-----------------------| | `Inner`, `Left` | Right input | | `Right` | Left input | | `LeftSemi`, `LeftAnti` | Left input | | `RightSemi`, `RightAnti`| Right input | | `LeftMark` | Right input | | `RightMark` | Left input | | `Full` | Not supported | Non-equi join predicates do **not** generate dynamic filters; they require range analysis and cross-conjunct reasoning (future work). Composite predicates only derive filters from their equi-conjuncts, and rows with `NULL` join keys (see [`crate::NullEquality::NullEqualsNothing`]) do not contribute filter values. Enabling `optimizer.filter_null_join_keys` can remove such rows up front. Pushdown is effective only when the file source supports predicate pushdown (e.g. Parquet) and `execution.parquet.pushdown_filters` is `true`; formats without predicate pushdown (CSV/JSON) see no benefit. See the upgrade guide for additional details and examples. For example, `SELECT * FROM fact LEFT JOIN dim ON fact.id = dim.id WHERE dim.region = 'US'` will only read `fact` rows whose `id` values match `dim` rows where `region = 'US'`. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible datafusion.optimizer.expand_views_at_output false When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. From 90c15f4afaeff8c14573405ab2e6d5600353138e Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Thu, 14 Aug 2025 10:58:12 +0000 Subject: [PATCH 60/72] Bless SLT outputs via CI (--complete) --- .../test_files/tpch/plans/q10.slt.part | 6 +++--- .../test_files/tpch/plans/q11.slt.part | 8 ++++---- .../test_files/tpch/plans/q12.slt.part | 2 +- .../test_files/tpch/plans/q13.slt.part | 2 +- .../test_files/tpch/plans/q14.slt.part | 2 +- .../test_files/tpch/plans/q15.slt.part | 4 ++-- .../test_files/tpch/plans/q16.slt.part | 4 ++-- .../test_files/tpch/plans/q17.slt.part | 4 ++-- .../test_files/tpch/plans/q18.slt.part | 6 +++--- .../test_files/tpch/plans/q19.slt.part | 2 +- .../test_files/tpch/plans/q2.slt.part | 16 ++++++++-------- .../test_files/tpch/plans/q20.slt.part | 8 ++++---- .../test_files/tpch/plans/q21.slt.part | 10 +++++----- .../test_files/tpch/plans/q22.slt.part | 2 +- .../test_files/tpch/plans/q3.slt.part | 4 ++-- .../test_files/tpch/plans/q4.slt.part | 2 +- .../test_files/tpch/plans/q5.slt.part | 10 +++++----- .../test_files/tpch/plans/q7.slt.part | 10 +++++----- .../test_files/tpch/plans/q8.slt.part | 14 +++++++------- .../test_files/tpch/plans/q9.slt.part | 10 +++++----- 20 files changed, 63 insertions(+), 63 deletions(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 04de9153a047..7c735a61e343 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -77,15 +77,15 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10], probe_side=Right, probe_keys=0 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] +13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10], probe_side=Right, probe_keys=0 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7], probe_side=Right, probe_keys=0 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 20)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index a6225daae436..7d4162b7c3f4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -82,11 +82,11 @@ physical_plan 07)------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2], probe_side=Right, probe_keys=0 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5], probe_side=Right, probe_keys=0 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 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_availqty, ps_supplycost], file_type=csv, has_header=false @@ -105,11 +105,11 @@ physical_plan 30)----------CoalescePartitionsExec 31)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] 32)--------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1], probe_side=Right, probe_keys=0 34)------------------CoalesceBatchesExec: target_batch_size=8192 35)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 36)----------------------CoalesceBatchesExec: target_batch_size=8192 -37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4], probe_side=Right, probe_keys=0 38)--------------------------CoalesceBatchesExec: target_batch_size=8192 39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 40)------------------------------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_suppkey, ps_availqty, ps_supplycost], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index f7344daed8c7..0f5af11eb2fd 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -67,7 +67,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] 08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3], probe_side=Right, probe_keys=0 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 96f3bd6edf32..bbbe1aaeebb0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -63,7 +63,7 @@ physical_plan 08)--------------ProjectionExec: expr=[count(orders.o_orderkey)@1 as c_count] 09)----------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[count(orders.o_orderkey)] 10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] +11)--------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1], probe_side=Right, probe_keys=0 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index 8d8dd68c3d7b..fe68bfb99865 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -47,7 +47,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------ProjectionExec: expr=[l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as __common_expr_1, p_type@2 as p_type] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4], probe_side=Right, probe_keys=0 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 10)------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 0636a033b25a..7bf7d40ff5e9 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -74,7 +74,7 @@ physical_plan 01)SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] 02)--SortExec: expr=[s_suppkey@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(max(revenue0.total_revenue)@0, total_revenue@4)], projection=[s_suppkey@1, s_name@2, s_address@3, s_phone@4, total_revenue@5] +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(max(revenue0.total_revenue)@0, total_revenue@4)], projection=[s_suppkey@1, s_name@2, s_address@3, s_phone@4, total_revenue@5], probe_side=Right, probe_keys=0 05)--------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] 06)----------CoalescePartitionsExec 07)------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] @@ -87,7 +87,7 @@ physical_plan 14)--------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] 15)----------------------------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_suppkey, l_extendedprice, l_discount, l_shipdate], file_type=csv, has_header=false 16)--------CoalesceBatchesExec: target_batch_size=8192 -17)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] +17)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5], probe_side=Right, probe_keys=0 18)------------CoalesceBatchesExec: target_batch_size=8192 19)--------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 20)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index 39f99a0fcf98..fca3a395398c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -77,11 +77,11 @@ physical_plan 10)------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] +13)------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)], probe_side=Right, probe_keys=0 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5], probe_side=Right, probe_keys=0 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 20)--------------------------------------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], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 51a0d096428c..900c8e04251c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -56,9 +56,9 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice)] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1] +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * avg(lineitem.l_quantity)@1, projection=[l_extendedprice@1], probe_side=Right, probe_keys=0 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3], probe_side=Right, probe_keys=0 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 11)--------------------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_partkey, l_quantity, l_extendedprice], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 55da5371671e..9a1579395d77 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -74,13 +74,13 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[sum(lineitem.l_quantity)] 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] +08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)], probe_side=Right, probe_keys=0 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6], probe_side=Right, probe_keys=0 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5], probe_side=Right, probe_keys=0 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 4cfbdc18ca50..1fef2c46dd33 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -69,7 +69,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND p_container@3 IN ([Literal { value: Utf8View("SM CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("SM PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_container@3 IN ([Literal { value: Utf8View("MED BAG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("MED PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_container@3 IN ([Literal { value: Utf8View("LG CASE"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG BOX"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PACK"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }, Literal { value: Utf8View("LG PKG"), field: Field { name: "lit", data_type: Utf8View, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} } }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3], probe_side=Right, probe_keys=0 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 09)----------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index b2e0fb0cd1cc..0e330c0de7ca 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -103,24 +103,24 @@ physical_plan 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)------CoalesceBatchesExec: target_batch_size=8192 -05)--------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)--------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], probe_side=Right, probe_keys=0 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------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] +09)----------------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], probe_side=Right, probe_keys=0 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------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] +13)------------------------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], probe_side=Right, probe_keys=0 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 16)------------------------------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] 17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------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] +18)----------------------------------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], probe_side=Right, probe_keys=0 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 20)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------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] +22)------------------------------------------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], probe_side=Right, probe_keys=0 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 @@ -152,15 +152,15 @@ physical_plan 51)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 52)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] 53)------------------------CoalesceBatchesExec: target_batch_size=8192 -54)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +54)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1], probe_side=Right, probe_keys=0 55)----------------------------CoalesceBatchesExec: target_batch_size=8192 56)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 57)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +58)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4], probe_side=Right, probe_keys=0 59)------------------------------------CoalesceBatchesExec: target_batch_size=8192 60)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 61)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4], probe_side=Right, probe_keys=0 63)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 64)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 65)------------------------------------------------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 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 0b994de411ea..f3ff7a099814 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -84,11 +84,11 @@ physical_plan 01)SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] 02)--SortExec: expr=[s_name@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] +04)------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2], probe_side=Right, probe_keys=0 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2], probe_side=Right, probe_keys=0 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -102,11 +102,11 @@ physical_plan 19)--------CoalesceBatchesExec: target_batch_size=8192 20)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 21)------------CoalesceBatchesExec: target_batch_size=8192 -22)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +22)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1], probe_side=Right, probe_keys=0 23)----------------CoalesceBatchesExec: target_batch_size=8192 24)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 25)--------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)], probe_side=Right, probe_keys=0 27)------------------------CoalesceBatchesExec: target_batch_size=8192 28)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 29)----------------------------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_availqty], file_type=csv, has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index e52171524007..839e790cd261 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -98,21 +98,21 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[count(Int64(1))] 08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] +09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0], probe_side=Right, probe_keys=0 10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 +11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, probe_side=Right, probe_keys=0 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3], probe_side=Right, probe_keys=0 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3], probe_side=Right, probe_keys=0 20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 21)----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] +23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4], probe_side=Right, probe_keys=0 24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index e9b533f2044f..f954754d7bb7 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -86,7 +86,7 @@ physical_plan 10)------------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > avg(customer.c_acctbal)@1 11)--------------------CoalescePartitionsExec 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] +13)------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2], probe_side=Right, probe_keys=0 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index d982ec32e954..f59cb4754592 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -66,11 +66,11 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5], probe_side=Right, probe_keys=0 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] +13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4], probe_side=Right, probe_keys=0 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index f7de3cd3c967..7f6b69dd56fb 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -61,7 +61,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[count(Int64(1))] 08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] +09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1], probe_side=Right, probe_keys=0 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index 15636056b871..a699c12ad6b4 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -75,23 +75,23 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2], probe_side=Right, probe_keys=0 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] +13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5], probe_side=Right, probe_keys=0 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5], probe_side=Right, probe_keys=0 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5], probe_side=Right, probe_keys=0 22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 23)--------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] +25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2], probe_side=Right, probe_keys=0 26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index 291d56e43f2d..f5d7dbdbaeac 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -93,23 +93,23 @@ physical_plan 07)------------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)] 08)--------------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] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------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] +10)------------------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], probe_side=Right, probe_keys=0 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------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] +14)--------------------------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], probe_side=Right, probe_keys=0 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6] +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6], probe_side=Right, probe_keys=0 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 20)--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6], probe_side=Right, probe_keys=0 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6], probe_side=Right, probe_keys=0 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 29)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 50171c528db6..a9a7b375a1b1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -99,31 +99,31 @@ physical_plan 07)------------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)] 08)--------------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] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------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] +10)------------------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], probe_side=Right, probe_keys=0 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------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] +14)--------------------------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], probe_side=Right, probe_keys=0 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6], probe_side=Right, probe_keys=0 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 20)--------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6], probe_side=Right, probe_keys=0 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6], probe_side=Right, probe_keys=0 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] +30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5], probe_side=Right, probe_keys=0 31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 33)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] +34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5], probe_side=Right, probe_keys=0 35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 3b31c1bc2e8e..c40f2476f3b3 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -84,23 +84,23 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 08)--------------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] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------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] +10)------------------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], probe_side=Right, probe_keys=0 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------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] +14)--------------------------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], probe_side=Right, probe_keys=0 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9], probe_side=Right, probe_keys=0 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 20)--------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7], probe_side=Right, probe_keys=0 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6], probe_side=Right, probe_keys=0 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 From d07afea06f22327d183e7b14a1b3a44f43ecb829 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 11:03:11 +0800 Subject: [PATCH 61/72] feat(join): add preservation methods for join types and remove unused modules --- datafusion/common/src/join_type.rs | 57 ++++++++++ datafusion/common/src/joins/mod.rs | 22 ---- datafusion/common/src/joins/preservation.rs | 101 ------------------ datafusion/common/src/lib.rs | 1 - .../optimizer/src/filter_null_join_keys.rs | 5 +- datafusion/optimizer/src/push_down_filter.rs | 14 +-- .../physical-plan/src/joins/hash_join.rs | 25 +++-- 7 files changed, 80 insertions(+), 145 deletions(-) delete mode 100644 datafusion/common/src/joins/mod.rs delete mode 100644 datafusion/common/src/joins/preservation.rs diff --git a/datafusion/common/src/join_type.rs b/datafusion/common/src/join_type.rs index d9a1478f0238..77d75179da38 100644 --- a/datafusion/common/src/join_type.rs +++ b/datafusion/common/src/join_type.rs @@ -111,6 +111,63 @@ impl JoinType { | JoinType::RightAnti ) } + /// Returns true if the left side of this join preserves its input rows + /// for filters applied *after* the join. + #[inline] + pub const fn preserves_left_for_output_filters(self) -> bool { + matches!( + self, + JoinType::Inner + | JoinType::Left + | JoinType::LeftSemi + | JoinType::LeftAnti + | JoinType::LeftMark, + ) + } + + /// Returns true if the right side of this join preserves its input rows + /// for filters applied *after* the join. + #[inline] + pub const fn preserves_right_for_output_filters(self) -> bool { + matches!( + self, + JoinType::Inner + | JoinType::Right + | JoinType::RightSemi + | JoinType::RightAnti + | JoinType::RightMark, + ) + } + + /// Returns true if the left side of this join preserves its input rows + /// for filters in the join condition (ON-clause filters). + #[inline] + pub const fn preserves_left_for_on_filters(self) -> bool { + matches!( + self, + JoinType::Inner + | JoinType::Right + | JoinType::LeftSemi + | JoinType::RightSemi + | JoinType::RightAnti + | JoinType::RightMark, + ) + } + + /// Returns true if the right side of this join preserves its input rows + /// for filters in the join condition (ON-clause filters). + #[inline] + pub const fn preserves_right_for_on_filters(self) -> bool { + matches!( + self, + JoinType::Inner + | JoinType::Left + | JoinType::LeftSemi + | JoinType::RightSemi + | JoinType::LeftAnti + | JoinType::LeftMark, + ) + } } impl Display for JoinType { diff --git a/datafusion/common/src/joins/mod.rs b/datafusion/common/src/joins/mod.rs deleted file mode 100644 index f6892b4e9e6b..000000000000 --- a/datafusion/common/src/joins/mod.rs +++ /dev/null @@ -1,22 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Utilities shared between join implementations. - -pub mod preservation; - -pub use preservation::{preservation_for_on_filters, preservation_for_output_filters}; diff --git a/datafusion/common/src/joins/preservation.rs b/datafusion/common/src/joins/preservation.rs deleted file mode 100644 index 08e44555a01a..000000000000 --- a/datafusion/common/src/joins/preservation.rs +++ /dev/null @@ -1,101 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Helpers for reasoning about which sides of a [`JoinType`] preserve -//! their input rows. -//! **`preservation_for_output_filters` answers post‑join output filtering, whereas `preservation_for_on_filters` addresses ON‑clause feasibility.** - -use crate::JoinType; - -/// For a given [`JoinType`], determine whether each input of the join is -/// preserved for filters applied *after* the join. -/// -/// Row preservation means every output row can be traced back to a row from -/// that input. Non‑preserved sides may introduce additional NULL padded rows. -/// The table below visualises the behaviour (`✓` preserved, `✗` not preserved): -/// -/// ```text -/// left right -/// INNER ✓ ✓ -/// LEFT ✓ ✗ -/// RIGHT ✗ ✓ -/// FULL ✗ ✗ -/// LEFT SEMI ✓ ✗ -/// LEFT ANTI ✓ ✗ -/// LEFT MARK ✓ ✗ -/// RIGHT SEMI ✗ ✓ -/// RIGHT ANTI ✗ ✓ -/// RIGHT MARK ✗ ✓ -/// ``` -/// -/// The returned tuple `(left_preserved, right_preserved)` reports whether each -/// side of the join preserves its input rows under post‑join filtering. -#[inline] -pub const fn preservation_for_output_filters(join_type: JoinType) -> (bool, bool) { - match join_type { - JoinType::Inner => (true, true), - JoinType::Left => (true, false), - JoinType::Right => (false, true), - JoinType::Full => (false, false), - // For semi/anti joins the non-driving side cannot appear in the output. - JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => (true, false), - JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => (false, true), - } -} - -/// For a given [`JoinType`], determine whether each input of the join is -/// preserved for filters in the join condition (ON‑clause filters). -/// -/// Filters on ON‑clause expressions may only reference sides that are -/// preserved; otherwise pushing the filter below the join could drop rows. -/// This table shows preservation for ON‑clause evaluation (`✓` preserved): -/// -/// ```text -/// left right -/// INNER ✓ ✓ -/// LEFT ✗ ✓ -/// RIGHT ✓ ✗ -/// FULL ✗ ✗ -/// LEFT SEMI ✓ ✓ -/// RIGHT SEMI ✓ ✓ -/// LEFT ANTI ✗ ✓ -/// RIGHT ANTI ✓ ✗ -/// LEFT MARK ✗ ✓ -/// RIGHT MARK ✓ ✗ -/// ``` -/// -/// The returned tuple `(left_preserved, right_preserved)` reports which sides -/// may safely participate in ON‑clause filtering. -#[inline] -pub const fn preservation_for_on_filters(join_type: JoinType) -> (bool, bool) { - match join_type { - JoinType::Inner => (true, true), - JoinType::Left => (false, true), - JoinType::Right => (true, false), - JoinType::Full => (false, false), - JoinType::LeftSemi | JoinType::RightSemi => (true, true), - JoinType::LeftAnti => (false, true), - JoinType::RightAnti => (true, false), - JoinType::LeftMark => (false, true), - JoinType::RightMark => (true, false), - } -} - -#[allow(unused_imports)] -pub(crate) use preservation_for_on_filters as on_lr_is_preserved; -#[allow(unused_imports)] -pub(crate) use preservation_for_output_filters as lr_is_preserved; diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 4378ae1503b4..3a558fa86789 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -47,7 +47,6 @@ pub mod file_options; pub mod format; pub mod hash_utils; pub mod instant; -pub mod joins; pub mod nested_struct; mod null_equality; pub mod parsers; diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index a14635f872e4..5cd20abb3e30 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -19,7 +19,6 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::joins::preservation_for_on_filters; use datafusion_common::tree_node::Transformed; use datafusion_common::{NullEquality, Result}; use datafusion_expr::utils::conjunction; @@ -54,8 +53,8 @@ impl OptimizerRule for FilterNullJoinKeys { if !join.on.is_empty() && join.null_equality == NullEquality::NullEqualsNothing => { - let (left_preserved, right_preserved) = - preservation_for_on_filters(join.join_type); + let left_preserved = join.join_type.preserves_left_for_on_filters(); + let right_preserved = join.join_type.preserves_right_for_on_filters(); let left_schema = join.left.schema(); let right_schema = join.right.schema(); diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 26878852628c..5a763d40a9ba 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -24,11 +24,7 @@ use arrow::datatypes::DataType; use indexmap::IndexSet; use itertools::Itertools; -// `preservation_for_output_filters` determines output-filter safety; `preservation_for_on_filters` -// governs ON-clause pushdown. -use datafusion_common::joins::{ - preservation_for_on_filters, preservation_for_output_filters, -}; +// JoinType's preservation helpers determine output-filter safety and ON-clause pushdown. use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -389,8 +385,8 @@ fn push_down_all_join( ) -> Result> { let is_inner_join = join.join_type == JoinType::Inner; // Get pushable predicates from current optimizer state - let (left_preserved, right_preserved) = - preservation_for_output_filters(join.join_type); + let left_preserved = join.join_type.preserves_left_for_output_filters(); + let right_preserved = join.join_type.preserves_right_for_output_filters(); // The predicates can be divided to three categories: // 1) can push through join to its children(left or right) @@ -427,8 +423,8 @@ fn push_down_all_join( } let mut on_filter_join_conditions = vec![]; - let (on_left_preserved, on_right_preserved) = - preservation_for_on_filters(join.join_type); + let on_left_preserved = join.join_type.preserves_left_for_on_filters(); + let on_right_preserved = join.join_type.preserves_right_for_on_filters(); if !on_filter.is_empty() { for on in on_filter { diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 67f1b3523c6d..6195ea042bf3 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -75,9 +75,8 @@ use arrow::util::bit_util; use datafusion_common::config::ConfigOptions; use datafusion_common::utils::memory::estimate_memory_size; use datafusion_common::{ - internal_datafusion_err, internal_err, - joins::{preservation_for_on_filters, preservation_for_output_filters}, - plan_err, project_schema, JoinSide, JoinType, NullEquality, Result, ScalarValue, + internal_datafusion_err, internal_err, plan_err, project_schema, JoinSide, JoinType, + NullEquality, Result, ScalarValue, }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; @@ -101,8 +100,10 @@ use parking_lot::Mutex; /// that only rows capable of satisfying the `ON` clause are evaluated. #[inline] fn dynamic_filter_side(join_type: JoinType) -> JoinSide { - let (left_preserved, right_preserved) = preservation_for_output_filters(join_type); - let (on_left_preserved, on_right_preserved) = preservation_for_on_filters(join_type); + let left_preserved = join_type.preserves_left_for_output_filters(); + let right_preserved = join_type.preserves_right_for_output_filters(); + let on_left_preserved = join_type.preserves_left_for_on_filters(); + let on_right_preserved = join_type.preserves_right_for_on_filters(); match ( left_preserved, right_preserved, @@ -1085,8 +1086,8 @@ impl ExecutionPlan for HashJoinExec { parent_filters: Vec>, config: &ConfigOptions, ) -> Result { - let (left_preserved, right_preserved) = - preservation_for_output_filters(self.join_type); + let left_preserved = self.join_type.preserves_left_for_output_filters(); + let right_preserved = self.join_type.preserves_right_for_output_filters(); let dynamic_target = dynamic_filter_side(self.join_type); // Prepare a single vector of unsupported predicates to avoid @@ -2058,8 +2059,14 @@ mod tests { let table: Vec = cases .iter() .map(|jt| { - let lr = preservation_for_output_filters(*jt); - let on = preservation_for_on_filters(*jt); + let lr = ( + jt.preserves_left_for_output_filters(), + jt.preserves_right_for_output_filters(), + ); + let on = ( + jt.preserves_left_for_on_filters(), + jt.preserves_right_for_on_filters(), + ); format!("{jt:?}: lr={lr:?}, on_lr={on:?}") }) .collect(); From 01c366f170db5fbe710718adbe0f634ca4ab7dfd Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 12:41:56 +0800 Subject: [PATCH 62/72] docs(tests): enhance comments for join type handling and dynamic filter pushdown --- .../core/tests/physical_optimizer/enforce_distribution.rs | 6 ++++++ .../core/tests/physical_optimizer/filter_pushdown/mod.rs | 3 +++ 2 files changed, 9 insertions(+) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 81c41fa31a1e..558e5803473f 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -676,6 +676,12 @@ fn multi_hash_joins() -> Result<()> { let top_join = hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); + // Determine probe_side and key offset based on join type: + // - Right and Full joins include both sides, so 'b1' offset is after left columns (index 6). + // probe_side is Left for Right joins, omitted for Full. + // - RightSemi/RightAnti only return right side columns, so 'b1' offset resets to index 1. + // probe_side is Left (semi/anti use Left as probe to filter right rows). + // - All other join types use probe_side=Right and standard offsets. let top_join_plan = match join_type { JoinType::Right => format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)], probe_side=Left, probe_keys=0"), diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 0f328821903e..cbf4fb1201cd 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -826,6 +826,9 @@ async fn test_topk_dynamic_filter_pushdown_multi_column_sort() { ]; assert_batches_eq!(expected, &[res]); // Now check what our filter looks like + // `filter_keys=2` indicates the dynamic filter was generated from two sort keys + // (here: `b@1 ASC NULLS LAST` and `a@0 DESC`) — TopK uses those two keys to + // build the predicate, so the plan shows `filter_keys=2`. insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" From bafcf2901ce09ded5d25eb995ba1780135c187c6 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 13:24:30 +0800 Subject: [PATCH 63/72] test: enhance dynamic filter pushdown tests for hash joins --- .../physical_optimizer/filter_pushdown/mod.rs | 102 +----------------- 1 file changed, 3 insertions(+), 99 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index cbf4fb1201cd..3cb19ff6a299 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -941,6 +941,9 @@ async fn test_hashjoin_dynamic_filter_pushdown() { stream.next().await.unwrap().unwrap(); // Now check what our filter looks like + // `probe_keys=2` indicates the dynamic filter was derived from two probe-side + // join key columns (here `a` and `b`). It verifies the optimizer generated a + // probe-side predicate that constrains both join keys from the small build side. insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" @@ -1162,105 +1165,6 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { .with_batches(t1_batches) .build(); - // t2: larger table (will be probe side of inner join, build side of outer join) - let t2_batches = vec![record_batch!( - ("b", Utf8, ["aa", "ab", "ac", "ad", "ae"]), - ("c", Utf8, ["ca", "cb", "cc", "cd", "ce"]), - ("y", Float64, [1.0, 2.0, 3.0, 4.0, 5.0]) - ) - .unwrap()]; - let t2_schema = Arc::new(Schema::new(vec![ - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Utf8, false), - Field::new("y", DataType::Float64, false), - ])); - let t2_scan = TestScanBuilder::new(Arc::clone(&t2_schema)) - .with_support(true) - .with_batches(t2_batches) - .build(); - - // t3: largest table (will be probe side of inner join) - let t3_batches = vec![record_batch!( - ("d", Utf8, ["ca", "cb", "cc", "cd", "ce", "cf", "cg", "ch"]), - ("z", Float64, [1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0]) - ) - .unwrap()]; - let t3_schema = Arc::new(Schema::new(vec![ - Field::new("d", DataType::Utf8, false), - Field::new("z", DataType::Float64, false), - ])); - let t3_scan = TestScanBuilder::new(Arc::clone(&t3_schema)) - .with_support(true) - .with_batches(t3_batches) - .build(); - - // Create nested join structure: - // Join (t1.a = t2.b) - // / \ - // t1 Join(t2.c = t3.d) - // / \ - // t2 t3 - - // First create inner join: t2.c = t3.d - let inner_join_on = - vec![(col("c", &t2_schema).unwrap(), col("d", &t3_schema).unwrap())]; - let inner_join = Arc::new( - HashJoinExec::try_new( - t2_scan, - t3_scan, - inner_join_on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - ) - .unwrap(), - ); - - // Then create outer join: t1.a = t2.b (from inner join result) - let outer_join_on = vec![( - col("a", &t1_schema).unwrap(), - col("b", &inner_join.schema()).unwrap(), - )]; - let outer_join = Arc::new( - HashJoinExec::try_new( - t1_scan, - inner_join as Arc, - outer_join_on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - ) - .unwrap(), - ) as Arc; - - // Test that dynamic filters are pushed down correctly through nested joins - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&outer_join), FilterPushdown::new_post_optimization(), true), - @r" - OptimizationTest: - input: - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], probe_side=Right, probe_keys=0 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true, predicate= - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_side=Right, probe_keys=0 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate= - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate= - output: - Ok: - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], probe_side=Right, probe_keys=0 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true, predicate= - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_side=Right, probe_keys=0 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ true ] - ", - ); - - // Execute the plan to verify the dynamic filters are properly updated - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; let plan = FilterPushdown::new_post_optimization() .optimize(outer_join, &config) From 1395b4939cbfb297436afa937a8c8cee85187286 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 13:29:05 +0800 Subject: [PATCH 64/72] refactor(tests): remove redundant hash join parent filter pushdown test --- .../physical_optimizer/filter_pushdown/mod.rs | 101 ------------------ 1 file changed, 101 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 3cb19ff6a299..50ecd9924d8a 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1204,107 +1204,6 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { assert_projection(&inner.right, &["d", "z"]); } -#[tokio::test] -async fn test_hashjoin_parent_filter_pushdown() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create build side with limited values - let build_batches = vec![record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8, ["ba", "bb"]), - ("c", Float64, [1.0, 2.0]) - ) - .unwrap()]; - let build_side_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Float64, false), - ])); - let build_scan = TestScanBuilder::new(Arc::clone(&build_side_schema)) - .with_support(true) - .with_batches(build_batches) - .build(); - - // Create probe side with more values - let probe_batches = vec![record_batch!( - ("d", Utf8, ["aa", "ab", "ac", "ad"]), - ("e", Utf8, ["ba", "bb", "bc", "bd"]), - ("f", Float64, [1.0, 2.0, 3.0, 4.0]) - ) - .unwrap()]; - let probe_side_schema = Arc::new(Schema::new(vec![ - Field::new("d", DataType::Utf8, false), - Field::new("e", DataType::Utf8, false), - Field::new("f", DataType::Float64, false), - ])); - let probe_scan = TestScanBuilder::new(Arc::clone(&probe_side_schema)) - .with_support(true) - .with_batches(probe_batches) - .build(); - - // Create HashJoinExec - let on = vec![( - col("a", &build_side_schema).unwrap(), - col("d", &probe_side_schema).unwrap(), - )]; - let join = Arc::new( - HashJoinExec::try_new( - build_scan, - probe_scan, - on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - ) - .unwrap(), - ); - - // Create filters that can be pushed down to different sides - // We need to create filters in the context of the join output schema - let join_schema = join.schema(); - - // Filter on build side column: a = 'aa' - let left_filter = col_lit_predicate("a", "aa", &join_schema); - // Filter on probe side column: e = 'ba' - let right_filter = col_lit_predicate("e", "ba", &join_schema); - // Filter that references both sides: a = d (should not be pushed down) - let cross_filter = Arc::new(BinaryExpr::new( - col("a", &join_schema).unwrap(), - Operator::Eq, - col("d", &join_schema).unwrap(), - )) as Arc; - - let filter = - Arc::new(FilterExec::try_new(left_filter, Arc::clone(&join) as _).unwrap()); - let filter = Arc::new(FilterExec::try_new(right_filter, filter).unwrap()); - let plan = Arc::new(FilterExec::try_new(cross_filter, filter).unwrap()) - as Arc; - - // Test that filters are pushed down correctly to each side of the join - insta::assert_snapshot!( - OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new(), true), - @r" - OptimizationTest: - input: - - FilterExec: a@0 = d@3 - - FilterExec: e@4 = ba - - FilterExec: a@0 = aa - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate= - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate= - output: - Ok: - - FilterExec: a@0 = d@3 - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, d@0)], probe_side=Right, probe_keys=0 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = aa - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate=e@1 = ba - " - ); -} - fn build_join_with_dynamic_filter( join_type: JoinType, left_support: bool, From 4799c019c1a1c26fdb498b63865cd9d8a98caab9 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 15:49:21 +0800 Subject: [PATCH 65/72] refactor(tests): consolidate tests --- .../physical_optimizer/filter_pushdown/mod.rs | 369 ++++++++---------- 1 file changed, 161 insertions(+), 208 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 50ecd9924d8a..d811c1c7712d 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -59,6 +59,7 @@ use datafusion_physical_plan::{ use futures::StreamExt; use object_store::{memory::InMemory, ObjectStore}; +use rstest::rstest; use util::{format_plan_for_test, OptimizationTest, TestNode, TestScanBuilder}; mod util; @@ -1164,7 +1165,75 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { .with_support(true) .with_batches(t1_batches) .build(); + // t2 and t3: larger tables joined together on (c = d) + let t2_batches = vec![record_batch!( + ("b", Utf8, ["aa", "ab", "ac", "ad", "ae"]), + ("c", Utf8, ["ca", "cb", "cc", "cd", "ce"]), + ("y", Float64, [1.0, 2.0, 3.0, 4.0, 5.0]) + ) + .unwrap()]; + let t2_schema = Arc::new(Schema::new(vec![ + Field::new("b", DataType::Utf8, false), + Field::new("c", DataType::Utf8, false), + Field::new("y", DataType::Float64, false), + ])); + let t2_scan = TestScanBuilder::new(Arc::clone(&t2_schema)) + .with_support(true) + .with_batches(t2_batches) + .build(); + + let t3_batches = vec![record_batch!( + ("d", Utf8, ["ca", "cb", "cc", "cd", "ce"]), + ("z", Float64, [1.0, 2.0, 3.0, 4.0, 5.0]) + ) + .unwrap()]; + let t3_schema = Arc::new(Schema::new(vec![ + Field::new("d", DataType::Utf8, false), + Field::new("z", DataType::Float64, false), + ])); + let t3_scan = TestScanBuilder::new(Arc::clone(&t3_schema)) + .with_support(true) + .with_batches(t3_batches) + .build(); + + // Inner join t2 and t3 on c = d + let inner_on = vec![(col("c", &t2_schema).unwrap(), col("d", &t3_schema).unwrap())]; + let inner_join = Arc::new( + HashJoinExec::try_new( + t2_scan, + t3_scan, + inner_on, + None, + &JoinType::Inner, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ) + .unwrap(), + ) as Arc; + + // Outer join t1 with the inner join on a = b + let inner_schema = inner_join.schema(); + let outer_on = vec![( + col("a", &t1_schema).unwrap(), + col("b", &inner_schema).unwrap(), + )]; + let outer_join = Arc::new( + HashJoinExec::try_new( + t1_scan, + inner_join, + outer_on, + None, + &JoinType::Inner, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ) + .unwrap(), + ) as Arc; + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; let plan = FilterPushdown::new_post_optimization() .optimize(outer_join, &config) @@ -1268,8 +1337,44 @@ fn build_join_with_dynamic_filter( ) as Arc } +fn assert_dynamic_filter_location(formatted: &str, join_type: &JoinType) { + match join_type { + JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { + assert_contains!( + formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_contains!( + formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" + ); + } + JoinType::Right + | JoinType::RightSemi + | JoinType::RightAnti + | JoinType::RightMark => { + assert_contains!( + formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_contains!( + formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" + ); + } + _ => unreachable!(), + } +} + +#[rstest( + join_type, + case::inner(JoinType::Inner), + case::left(JoinType::Left), + case::right(JoinType::Right), + case::full(JoinType::Full) +)] #[tokio::test] -async fn test_hashjoin_non_equi_predicate_no_dynamic_filter() { +async fn test_hashjoin_non_equi_predicate_no_dynamic_filter(join_type: JoinType) { // Non-equi join predicates like `l.a > r.a` are not supported by HashJoinExec // and thus cannot produce dynamic filters. let left_batches = vec![record_batch!(("a", Utf8, ["aa", "ab"])).unwrap()]; @@ -1304,7 +1409,7 @@ async fn test_hashjoin_non_equi_predicate_no_dynamic_filter() { right_scan, vec![], Some(join_filter), - &JoinType::Inner, + &join_type, None, PartitionMode::Partitioned, datafusion_common::NullEquality::NullEqualsNothing, @@ -1313,58 +1418,11 @@ async fn test_hashjoin_non_equi_predicate_no_dynamic_filter() { assert_not_contains!(err.to_string(), "non-equi"); } +#[rstest(join_type, case::left(JoinType::Left), case::right(JoinType::Right))] #[tokio::test] -async fn test_hashjoin_left_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::Left, - true, - true, - PartitionMode::Partitioned, - ); - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" - ); - assert_contains!(&formatted, "probe_keys=0"); -} - -#[tokio::test] -async fn test_hashjoin_left_dynamic_filter_pushdown_disabled() { - let plan = build_join_with_dynamic_filter( - JoinType::Left, - true, - true, - PartitionMode::Partitioned, - ); - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = false; - config.optimizer.enable_dynamic_filter_pushdown = false; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let formatted = format_plan_for_test(&plan); - assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); -} - -#[tokio::test] -async fn test_hashjoin_right_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::Right, - true, - true, - PartitionMode::Partitioned, - ); +async fn test_hashjoin_outer_dynamic_filter_pushdown(join_type: JoinType) { + let plan = + build_join_with_dynamic_filter(join_type, true, true, PartitionMode::Partitioned); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1372,25 +1430,15 @@ async fn test_hashjoin_right_dynamic_filter_pushdown() { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" - ); + assert_dynamic_filter_location(&formatted, &join_type); assert_contains!(&formatted, "probe_keys=0"); } +#[rstest(join_type, case::left(JoinType::Left), case::right(JoinType::Right))] #[tokio::test] -async fn test_hashjoin_right_dynamic_filter_pushdown_disabled() { - let plan = build_join_with_dynamic_filter( - JoinType::Right, - true, - true, - PartitionMode::Partitioned, - ); +async fn test_hashjoin_outer_dynamic_filter_pushdown_disabled(join_type: JoinType) { + let plan = + build_join_with_dynamic_filter(join_type, true, true, PartitionMode::Partitioned); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = false; config.optimizer.enable_dynamic_filter_pushdown = false; @@ -1427,40 +1475,15 @@ async fn test_hashjoin_left_dynamic_filter_pushdown_collect_left() { assert_contains!(&formatted, "probe_keys=0"); } +#[rstest( + join_type, + case::left(JoinType::LeftSemi), + case::right(JoinType::RightSemi) +)] #[tokio::test] -async fn test_hashjoin_left_semi_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::LeftSemi, - true, - true, - PartitionMode::Partitioned, - ); - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" - ); - assert_contains!(&formatted, "probe_keys=0"); -} - -#[tokio::test] -async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::LeftAnti, - true, - true, - PartitionMode::Partitioned, - ); +async fn test_hashjoin_semi_dynamic_filter_pushdown(join_type: JoinType) { + let plan = + build_join_with_dynamic_filter(join_type, true, true, PartitionMode::Partitioned); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1468,25 +1491,19 @@ async fn test_hashjoin_left_anti_dynamic_filter_pushdown() { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" - ); + assert_dynamic_filter_location(&formatted, &join_type); assert_contains!(&formatted, "probe_keys=0"); } +#[rstest( + join_type, + case::left(JoinType::LeftAnti), + case::right(JoinType::RightAnti) +)] #[tokio::test] -async fn test_hashjoin_left_mark_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::LeftMark, - true, - true, - PartitionMode::Partitioned, - ); +async fn test_hashjoin_anti_dynamic_filter_pushdown(join_type: JoinType) { + let plan = + build_join_with_dynamic_filter(join_type, true, true, PartitionMode::Partitioned); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1494,25 +1511,19 @@ async fn test_hashjoin_left_mark_dynamic_filter_pushdown() { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" - ); + assert_dynamic_filter_location(&formatted, &join_type); assert_contains!(&formatted, "probe_keys=0"); } +#[rstest( + join_type, + case::left(JoinType::LeftMark), + case::right(JoinType::RightMark) +)] #[tokio::test] -async fn test_hashjoin_right_mark_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::RightMark, - true, - true, - PartitionMode::Partitioned, - ); +async fn test_hashjoin_mark_dynamic_filter_pushdown(join_type: JoinType) { + let plan = + build_join_with_dynamic_filter(join_type, true, true, PartitionMode::Partitioned); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; config.optimizer.enable_dynamic_filter_pushdown = true; @@ -1520,67 +1531,10 @@ async fn test_hashjoin_right_mark_dynamic_filter_pushdown() { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" - ); + assert_dynamic_filter_location(&formatted, &join_type); assert_contains!(&formatted, "probe_keys=0"); } -#[tokio::test] -async fn test_hashjoin_right_semi_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::RightSemi, - true, - true, - PartitionMode::Partitioned, - ); - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" - ); -} - -#[tokio::test] -async fn test_hashjoin_right_anti_dynamic_filter_pushdown() { - let plan = build_join_with_dynamic_filter( - JoinType::RightAnti, - true, - true, - PartitionMode::Partitioned, - ); - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=" - ); -} - #[tokio::test] async fn test_hashjoin_full_dynamic_filter_no_pushdown() { let plan = build_join_with_dynamic_filter( @@ -1599,13 +1553,26 @@ async fn test_hashjoin_full_dynamic_filter_no_pushdown() { assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); } +#[rstest( + join_type, + left_support, + right_support, + partition_mode, + case::left(JoinType::Left, true, false, PartitionMode::CollectLeft), + case::right(JoinType::Right, false, true, PartitionMode::Auto) +)] #[tokio::test] -async fn test_hashjoin_dynamic_filter_pushdown_unsupported() { +async fn test_hashjoin_dynamic_filter_pushdown_unsupported( + join_type: JoinType, + left_support: bool, + right_support: bool, + partition_mode: PartitionMode, +) { let plan = build_join_with_dynamic_filter( - JoinType::Left, - true, - false, - PartitionMode::CollectLeft, + join_type, + left_support, + right_support, + partition_mode, ); let mut config = ConfigOptions::default(); config.execution.parquet.pushdown_filters = true; @@ -1617,20 +1584,6 @@ async fn test_hashjoin_dynamic_filter_pushdown_unsupported() { assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); } -#[tokio::test] -async fn test_hashjoin_dynamic_filter_pushdown_unsupported_left() { - let plan = - build_join_with_dynamic_filter(JoinType::Right, false, true, PartitionMode::Auto); - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(plan, &config) - .unwrap(); - let formatted = format_plan_for_test(&plan); - assert_not_contains!(formatted, "DynamicFilterPhysicalExpr"); -} - /// Integration test for dynamic filter pushdown with TopK. /// We use an integration test because there are complex interactions in the optimizer rules /// that the unit tests applying a single optimizer rule do not cover. From b2cc0a5da289cfa19dd04ff6b1c9970e1c4bcf89 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 18:09:11 +0800 Subject: [PATCH 66/72] refactor(tests): clean up imports and remove unused join type references in filter pushdown tests --- .../physical_optimizer/filter_pushdown/mod.rs | 32 ++++--------------- 1 file changed, 6 insertions(+), 26 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index d811c1c7712d..59a8587477d4 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. -use std::sync::{Arc, LazyLock}; - use arrow::{ - array::record_batch, + array::{record_batch, ArrayRef, Float64Array, Int32Array}, datatypes::{DataType, Field, Schema, SchemaRef}, + record_batch::RecordBatch, util::pretty::pretty_format_batches, }; use arrow_schema::SortOptions; @@ -41,9 +40,9 @@ use datafusion_expr::ScalarUDF; use datafusion_functions::math::random::RandomFunc; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::{ - aggregate::AggregateExprBuilder, Partitioning, ScalarFunctionExpr, + aggregate::AggregateExprBuilder, expressions::col, LexOrdering, Partitioning, + PhysicalSortExpr, ScalarFunctionExpr, }; -use datafusion_physical_expr::{expressions::col, LexOrdering, PhysicalSortExpr}; use datafusion_physical_optimizer::{ filter_pushdown::FilterPushdown, PhysicalOptimizerRule, }; @@ -60,6 +59,8 @@ use datafusion_physical_plan::{ use futures::StreamExt; use object_store::{memory::InMemory, ObjectStore}; use rstest::rstest; +use std::sync::{Arc, LazyLock}; +use tokio::time::{timeout, Duration}; use util::{format_plan_for_test, OptimizationTest, TestNode, TestScanBuilder}; mod util; @@ -168,9 +169,6 @@ fn test_pushdown_into_scan_with_config_options() { #[tokio::test] async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - // Create build side with limited values let build_batches = vec![record_batch!( ("a", Utf8, ["aa", "ab"]), @@ -287,9 +285,6 @@ async fn test_dynamic_filter_pushdown_through_hash_join_with_topk() { // Dynamic filters arise in cases such as nested inner joins or TopK -> HashJoinExec -> Scan setups. #[tokio::test] async fn test_static_filter_pushdown_through_hash_join() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - // Create build side with limited values let build_batches = vec![record_batch!( ("a", Utf8, ["aa", "ab"]), @@ -843,9 +838,6 @@ async fn test_topk_dynamic_filter_pushdown_multi_column_sort() { #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - // Create build side with limited values let build_batches = vec![record_batch!( ("a", Utf8, ["aa", "ab"]), @@ -965,9 +957,6 @@ async fn test_hashjoin_dynamic_filter_pushdown() { #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - // Build side containing only null join keys let build_batches = vec![record_batch!( ("a", Utf8, [None::<&str>, None]), @@ -1060,12 +1049,6 @@ async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { - use arrow::array::{ArrayRef, Float64Array, Int32Array}; - use arrow::record_batch::RecordBatch; - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - use tokio::time::{timeout, Duration}; - // Generate large key sets to watch for planning regressions let size = 10_000; let build_schema = Arc::new(Schema::new(vec![ @@ -1148,9 +1131,6 @@ async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { #[tokio::test] async fn test_nested_hashjoin_dynamic_filter_pushdown() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - // Create test data for three tables: t1, t2, t3 // t1: small table with limited values (will be build side of outer join) let t1_batches = From 7a3c6dbc76931550ea8873f0e0759b9136ea8189 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 18:19:48 +0800 Subject: [PATCH 67/72] Rearrange tests to minimize diff --- .../physical_optimizer/filter_pushdown/mod.rs | 248 +++++++++--------- 1 file changed, 124 insertions(+), 124 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 59a8587477d4..10419d5b5b4f 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -955,6 +955,130 @@ async fn test_hashjoin_dynamic_filter_pushdown() { assert_projection(&join.right, &["a", "b", "e"]); } +#[tokio::test] +async fn test_nested_hashjoin_dynamic_filter_pushdown() { + // Create test data for three tables: t1, t2, t3 + // t1: small table with limited values (will be build side of outer join) + let t1_batches = + vec![ + record_batch!(("a", Utf8, ["aa", "ab"]), ("x", Float64, [1.0, 2.0])).unwrap(), + ]; + let t1_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, false), + Field::new("x", DataType::Float64, false), + ])); + let t1_scan = TestScanBuilder::new(Arc::clone(&t1_schema)) + .with_support(true) + .with_batches(t1_batches) + .build(); + // t2 and t3: larger tables joined together on (c = d) + let t2_batches = vec![record_batch!( + ("b", Utf8, ["aa", "ab", "ac", "ad", "ae"]), + ("c", Utf8, ["ca", "cb", "cc", "cd", "ce"]), + ("y", Float64, [1.0, 2.0, 3.0, 4.0, 5.0]) + ) + .unwrap()]; + let t2_schema = Arc::new(Schema::new(vec![ + Field::new("b", DataType::Utf8, false), + Field::new("c", DataType::Utf8, false), + Field::new("y", DataType::Float64, false), + ])); + let t2_scan = TestScanBuilder::new(Arc::clone(&t2_schema)) + .with_support(true) + .with_batches(t2_batches) + .build(); + + let t3_batches = vec![record_batch!( + ("d", Utf8, ["ca", "cb", "cc", "cd", "ce"]), + ("z", Float64, [1.0, 2.0, 3.0, 4.0, 5.0]) + ) + .unwrap()]; + let t3_schema = Arc::new(Schema::new(vec![ + Field::new("d", DataType::Utf8, false), + Field::new("z", DataType::Float64, false), + ])); + let t3_scan = TestScanBuilder::new(Arc::clone(&t3_schema)) + .with_support(true) + .with_batches(t3_batches) + .build(); + + // Inner join t2 and t3 on c = d + let inner_on = vec![(col("c", &t2_schema).unwrap(), col("d", &t3_schema).unwrap())]; + let inner_join = Arc::new( + HashJoinExec::try_new( + t2_scan, + t3_scan, + inner_on, + None, + &JoinType::Inner, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ) + .unwrap(), + ) as Arc; + + // Outer join t1 with the inner join on a = b + let inner_schema = inner_join.schema(); + let outer_on = vec![( + col("a", &t1_schema).unwrap(), + col("b", &inner_schema).unwrap(), + )]; + let outer_join = Arc::new( + HashJoinExec::try_new( + t1_scan, + inner_join, + outer_on, + None, + &JoinType::Inner, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ) + .unwrap(), + ) as Arc; + + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + let plan = FilterPushdown::new_post_optimization() + .optimize(outer_join, &config) + .unwrap(); + let config = SessionConfig::new().with_batch_size(10); + let session_ctx = SessionContext::new_with_config(config); + session_ctx.register_object_store( + ObjectStoreUrl::parse("test://").unwrap().as_ref(), + Arc::new(InMemory::new()), + ); + let state = session_ctx.state(); + let task_ctx = state.task_ctx(); + let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); + // Execute to populate the dynamic filters + stream.next().await.unwrap().unwrap(); + + // Verify that both the inner and outer join have updated dynamic filters + insta::assert_snapshot!( + format!("{}", format_plan_for_test(&plan)), + @r" + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], probe_filter=[b@0 >= aa AND b@0 <= ab], probe_side=Right, probe_keys=2 + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true, predicate= + - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_filter=[d@0 >= ca AND d@0 <= ce], probe_side=Right, probe_keys=5 + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ b@0 >= aa AND b@0 <= ab ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= ca AND d@0 <= ce ] + " + ); + + let formatted = format_plan_for_test(&plan); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(formatted, "probe_keys=0"); + + let outer = plan.as_any().downcast_ref::().unwrap(); + assert_projection(&outer.left, &["a", "x"]); + let inner = outer.right.as_any().downcast_ref::().unwrap(); + assert_projection(&inner.left, &["b", "c", "y"]); + assert_projection(&inner.right, &["d", "z"]); +} + #[tokio::test] async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { // Build side containing only null join keys @@ -1129,130 +1253,6 @@ async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { assert_projection(&join.right, &["a", "y"]); } -#[tokio::test] -async fn test_nested_hashjoin_dynamic_filter_pushdown() { - // Create test data for three tables: t1, t2, t3 - // t1: small table with limited values (will be build side of outer join) - let t1_batches = - vec![ - record_batch!(("a", Utf8, ["aa", "ab"]), ("x", Float64, [1.0, 2.0])).unwrap(), - ]; - let t1_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Utf8, false), - Field::new("x", DataType::Float64, false), - ])); - let t1_scan = TestScanBuilder::new(Arc::clone(&t1_schema)) - .with_support(true) - .with_batches(t1_batches) - .build(); - // t2 and t3: larger tables joined together on (c = d) - let t2_batches = vec![record_batch!( - ("b", Utf8, ["aa", "ab", "ac", "ad", "ae"]), - ("c", Utf8, ["ca", "cb", "cc", "cd", "ce"]), - ("y", Float64, [1.0, 2.0, 3.0, 4.0, 5.0]) - ) - .unwrap()]; - let t2_schema = Arc::new(Schema::new(vec![ - Field::new("b", DataType::Utf8, false), - Field::new("c", DataType::Utf8, false), - Field::new("y", DataType::Float64, false), - ])); - let t2_scan = TestScanBuilder::new(Arc::clone(&t2_schema)) - .with_support(true) - .with_batches(t2_batches) - .build(); - - let t3_batches = vec![record_batch!( - ("d", Utf8, ["ca", "cb", "cc", "cd", "ce"]), - ("z", Float64, [1.0, 2.0, 3.0, 4.0, 5.0]) - ) - .unwrap()]; - let t3_schema = Arc::new(Schema::new(vec![ - Field::new("d", DataType::Utf8, false), - Field::new("z", DataType::Float64, false), - ])); - let t3_scan = TestScanBuilder::new(Arc::clone(&t3_schema)) - .with_support(true) - .with_batches(t3_batches) - .build(); - - // Inner join t2 and t3 on c = d - let inner_on = vec![(col("c", &t2_schema).unwrap(), col("d", &t3_schema).unwrap())]; - let inner_join = Arc::new( - HashJoinExec::try_new( - t2_scan, - t3_scan, - inner_on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - ) - .unwrap(), - ) as Arc; - - // Outer join t1 with the inner join on a = b - let inner_schema = inner_join.schema(); - let outer_on = vec![( - col("a", &t1_schema).unwrap(), - col("b", &inner_schema).unwrap(), - )]; - let outer_join = Arc::new( - HashJoinExec::try_new( - t1_scan, - inner_join, - outer_on, - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - datafusion_common::NullEquality::NullEqualsNothing, - ) - .unwrap(), - ) as Arc; - - let mut config = ConfigOptions::default(); - config.execution.parquet.pushdown_filters = true; - config.optimizer.enable_dynamic_filter_pushdown = true; - let plan = FilterPushdown::new_post_optimization() - .optimize(outer_join, &config) - .unwrap(); - let config = SessionConfig::new().with_batch_size(10); - let session_ctx = SessionContext::new_with_config(config); - session_ctx.register_object_store( - ObjectStoreUrl::parse("test://").unwrap().as_ref(), - Arc::new(InMemory::new()), - ); - let state = session_ctx.state(); - let task_ctx = state.task_ctx(); - let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); - // Execute to populate the dynamic filters - stream.next().await.unwrap().unwrap(); - - // Verify that both the inner and outer join have updated dynamic filters - insta::assert_snapshot!( - format!("{}", format_plan_for_test(&plan)), - @r" - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@0)], probe_filter=[b@0 >= aa AND b@0 <= ab], probe_side=Right, probe_keys=2 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, x], file_type=test, pushdown_supported=true, predicate= - - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, d@0)], probe_filter=[d@0 >= ca AND d@0 <= ce], probe_side=Right, probe_keys=5 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[b, c, y], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ b@0 >= aa AND b@0 <= ab ] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, z], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr [ d@0 >= ca AND d@0 <= ce ] - " - ); - - let formatted = format_plan_for_test(&plan); - assert_contains!(&formatted, "probe_keys="); - assert_not_contains!(formatted, "probe_keys=0"); - - let outer = plan.as_any().downcast_ref::().unwrap(); - assert_projection(&outer.left, &["a", "x"]); - let inner = outer.right.as_any().downcast_ref::().unwrap(); - assert_projection(&inner.left, &["b", "c", "y"]); - assert_projection(&inner.right, &["d", "z"]); -} - fn build_join_with_dynamic_filter( join_type: JoinType, left_support: bool, From 08a0e29fc9a49a23035073ef5c49c8c08698175f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 18:55:18 +0800 Subject: [PATCH 68/72] docs(tests): add issue reference for dynamic filter pushdown test --- datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 10419d5b5b4f..4c5788de3db3 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1166,6 +1166,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { // TODO: NullEqualsNothing yields no matching keys, so the dynamic filter // becomes a tautology. + // https://github.com/apache/datafusion/issues/17206 let join = plan.as_any().downcast_ref::().unwrap(); assert_projection(&join.left, &["a", "b", "c"]); assert_projection(&join.right, &["a", "b", "e"]); From 7b6f7786ed4781e0f5c9b04bd6f170dc8b24ece2 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 19:07:54 +0800 Subject: [PATCH 69/72] refactor(tests): add dynamic filter location assertion and predicate accessor for TestSource --- .../physical_optimizer/filter_pushdown/mod.rs | 78 +++++++++++++++---- .../filter_pushdown/util.rs | 9 +++ 2 files changed, 74 insertions(+), 13 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 4c5788de3db3..8edb246d1bb5 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -1245,7 +1245,8 @@ async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { .expect("optimizer should finish in time") .unwrap(); let formatted = format_plan_for_test(&plan); - assert_contains!(&formatted, "DynamicFilterPhysicalExpr"); + // Programmatic check: ensure some child received a dynamic filter for Inner join + assert_dynamic_filter_location_plan(&plan, &JoinType::Inner); assert_contains!(&formatted, "probe_keys=0"); @@ -1347,6 +1348,63 @@ fn assert_dynamic_filter_location(formatted: &str, join_type: &JoinType) { } } +/// Programmatic check: inspect the `HashJoinExec` children and verify which +/// side received a pushed predicate in the underlying `TestSource`. +fn assert_dynamic_filter_location_plan( + plan: &Arc, + join_type: &JoinType, +) { + // Find top-level HashJoinExec + let join = plan + .as_any() + .downcast_ref::() + .expect("expected HashJoinExec"); + + // Helper to check whether a child has a pushed predicate + let child_has_predicate = |child: &Arc| -> bool { + if let Some(data_src) = child + .as_any() + .downcast_ref::( + ) { + if let Some((_, test_source)) = + data_src.downcast_to_file_source::() + { + return test_source.predicate().is_some(); + } + } + false + }; + + match join_type { + JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { + // For left-type joins the right side (probe) should receive the dynamic filter + assert!( + child_has_predicate(&join.right), + "expected predicate on right child" + ); + assert!( + !child_has_predicate(&join.left), + "expected no predicate on left child" + ); + } + JoinType::Right + | JoinType::RightSemi + | JoinType::RightAnti + | JoinType::RightMark => { + // For right-type joins the left side should receive the dynamic filter + assert!( + child_has_predicate(&join.left), + "expected predicate on left child" + ); + assert!( + !child_has_predicate(&join.right), + "expected no predicate on right child" + ); + } + _ => unreachable!(), + } +} + #[rstest( join_type, case::inner(JoinType::Inner), @@ -1411,7 +1469,7 @@ async fn test_hashjoin_outer_dynamic_filter_pushdown(join_type: JoinType) { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_dynamic_filter_location(&formatted, &join_type); + assert_dynamic_filter_location_plan(&plan, &join_type); assert_contains!(&formatted, "probe_keys=0"); } @@ -1445,14 +1503,8 @@ async fn test_hashjoin_left_dynamic_filter_pushdown_collect_left() { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" - ); - assert_contains!( - &formatted, - "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" - ); + // Programmatic check: verify the dynamic filter was pushed to the expected side + assert_dynamic_filter_location_plan(&plan, &JoinType::Left); assert_contains!(&formatted, "probe_keys=0"); } @@ -1472,7 +1524,7 @@ async fn test_hashjoin_semi_dynamic_filter_pushdown(join_type: JoinType) { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_dynamic_filter_location(&formatted, &join_type); + assert_dynamic_filter_location_plan(&plan, &join_type); assert_contains!(&formatted, "probe_keys=0"); } @@ -1492,7 +1544,7 @@ async fn test_hashjoin_anti_dynamic_filter_pushdown(join_type: JoinType) { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_dynamic_filter_location(&formatted, &join_type); + assert_dynamic_filter_location_plan(&plan, &join_type); assert_contains!(&formatted, "probe_keys=0"); } @@ -1512,7 +1564,7 @@ async fn test_hashjoin_mark_dynamic_filter_pushdown(join_type: JoinType) { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_dynamic_filter_location(&formatted, &join_type); + assert_dynamic_filter_location_plan(&plan, &join_type); assert_contains!(&formatted, "probe_keys=0"); } diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 885b32c092c1..8570ea8fdda3 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -125,6 +125,15 @@ impl TestSource { ..Default::default() } } + + /// Return the currently assigned predicate (if any). + /// + /// This is a test-only accessor used by assertions in the + /// `filter_pushdown` tests to verify that predicates were pushed into + /// the underlying file source. + pub fn predicate(&self) -> Option<&Arc> { + self.predicate.as_ref() + } } impl FileSource for TestSource { From 3023d88faecb4ff1b1bac79566a1f3b80a7bde1f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 19:22:07 +0800 Subject: [PATCH 70/72] fix tests --- .../physical_optimizer/filter_pushdown/mod.rs | 81 +++---------------- .../filter_pushdown/util.rs | 3 +- 2 files changed, 15 insertions(+), 69 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 8edb246d1bb5..2747b83f743d 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -934,9 +934,6 @@ async fn test_hashjoin_dynamic_filter_pushdown() { stream.next().await.unwrap().unwrap(); // Now check what our filter looks like - // `probe_keys=2` indicates the dynamic filter was derived from two probe-side - // join key columns (here `a` and `b`). It verifies the optimizer generated a - // probe-side predicate that constrains both join keys from the small build side. insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), @r" @@ -1245,8 +1242,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { .expect("optimizer should finish in time") .unwrap(); let formatted = format_plan_for_test(&plan); - // Programmatic check: ensure some child received a dynamic filter for Inner join - assert_dynamic_filter_location_plan(&plan, &JoinType::Inner); + assert_contains!(&formatted, "DynamicFilterPhysicalExpr"); assert_contains!(&formatted, "probe_keys=0"); @@ -1348,63 +1344,6 @@ fn assert_dynamic_filter_location(formatted: &str, join_type: &JoinType) { } } -/// Programmatic check: inspect the `HashJoinExec` children and verify which -/// side received a pushed predicate in the underlying `TestSource`. -fn assert_dynamic_filter_location_plan( - plan: &Arc, - join_type: &JoinType, -) { - // Find top-level HashJoinExec - let join = plan - .as_any() - .downcast_ref::() - .expect("expected HashJoinExec"); - - // Helper to check whether a child has a pushed predicate - let child_has_predicate = |child: &Arc| -> bool { - if let Some(data_src) = child - .as_any() - .downcast_ref::( - ) { - if let Some((_, test_source)) = - data_src.downcast_to_file_source::() - { - return test_source.predicate().is_some(); - } - } - false - }; - - match join_type { - JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { - // For left-type joins the right side (probe) should receive the dynamic filter - assert!( - child_has_predicate(&join.right), - "expected predicate on right child" - ); - assert!( - !child_has_predicate(&join.left), - "expected no predicate on left child" - ); - } - JoinType::Right - | JoinType::RightSemi - | JoinType::RightAnti - | JoinType::RightMark => { - // For right-type joins the left side should receive the dynamic filter - assert!( - child_has_predicate(&join.left), - "expected predicate on left child" - ); - assert!( - !child_has_predicate(&join.right), - "expected no predicate on right child" - ); - } - _ => unreachable!(), - } -} - #[rstest( join_type, case::inner(JoinType::Inner), @@ -1469,7 +1408,7 @@ async fn test_hashjoin_outer_dynamic_filter_pushdown(join_type: JoinType) { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_dynamic_filter_location_plan(&plan, &join_type); + assert_dynamic_filter_location(&formatted, &join_type); assert_contains!(&formatted, "probe_keys=0"); } @@ -1503,8 +1442,14 @@ async fn test_hashjoin_left_dynamic_filter_pushdown_collect_left() { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - // Programmatic check: verify the dynamic filter was pushed to the expected side - assert_dynamic_filter_location_plan(&plan, &JoinType::Left); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilterPhysicalExpr" + ); + assert_contains!( + &formatted, + "DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=" + ); assert_contains!(&formatted, "probe_keys=0"); } @@ -1524,7 +1469,7 @@ async fn test_hashjoin_semi_dynamic_filter_pushdown(join_type: JoinType) { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_dynamic_filter_location_plan(&plan, &join_type); + assert_dynamic_filter_location(&formatted, &join_type); assert_contains!(&formatted, "probe_keys=0"); } @@ -1544,7 +1489,7 @@ async fn test_hashjoin_anti_dynamic_filter_pushdown(join_type: JoinType) { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_dynamic_filter_location_plan(&plan, &join_type); + assert_dynamic_filter_location(&formatted, &join_type); assert_contains!(&formatted, "probe_keys=0"); } @@ -1564,7 +1509,7 @@ async fn test_hashjoin_mark_dynamic_filter_pushdown(join_type: JoinType) { .optimize(plan, &config) .unwrap(); let formatted = format_plan_for_test(&plan); - assert_dynamic_filter_location_plan(&plan, &join_type); + assert_dynamic_filter_location(&formatted, &join_type); assert_contains!(&formatted, "probe_keys=0"); } diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 8570ea8fdda3..7437801387c7 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -152,7 +152,8 @@ impl FileSource for TestSource { } fn as_any(&self) -> &dyn Any { - todo!("should not be called") + // Allow downcasting a FileSource to the concrete TestSource in tests. + self } fn with_batch_size(&self, batch_size: usize) -> Arc { From 4beb2f76993e8e1048f7c0dc144b08dbd9fc4b9b Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 19:51:41 +0800 Subject: [PATCH 71/72] refactor(tests): remove unused predicate accessor from TestSource --- .../tests/physical_optimizer/filter_pushdown/util.rs | 9 --------- 1 file changed, 9 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 7437801387c7..3876f3d43d8c 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -125,15 +125,6 @@ impl TestSource { ..Default::default() } } - - /// Return the currently assigned predicate (if any). - /// - /// This is a test-only accessor used by assertions in the - /// `filter_pushdown` tests to verify that predicates were pushed into - /// the underlying file source. - pub fn predicate(&self) -> Option<&Arc> { - self.predicate.as_ref() - } } impl FileSource for TestSource { From f999914478638caf39ecb3ccc8563e116b87ada8 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 15 Aug 2025 20:46:11 +0800 Subject: [PATCH 72/72] test: add async test for hash join with probe filter --- .../enforce_distribution.rs | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 558e5803473f..e028b12a558c 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -40,6 +40,7 @@ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::error::Result; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; +use datafusion_common::{assert_contains, assert_not_contains}; use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_expr::{JoinType, Operator}; @@ -740,6 +741,33 @@ fn multi_hash_joins() -> Result<()> { Ok(()) } +#[tokio::test] +async fn hash_join_with_probe_filter() -> Result<()> { + let left = crate::physical_optimizer::test_utils::mock_data()?; + let right = crate::physical_optimizer::test_utils::mock_data()?; + + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a", &right.schema()).unwrap()) as _, + )]; + let join = hash_join_exec(left, right, &join_on, &JoinType::Inner); + + let config = test_suite_default_config_options(); + let plan = EnforceDistribution::new().optimize(join, &config)?; + + let session_ctx = SessionContext::new_with_config(SessionConfig::new()); + let state = session_ctx.state(); + let task_ctx = state.task_ctx(); + let mut stream = plan.execute(0, Arc::clone(&task_ctx)).unwrap(); + use futures::StreamExt; + stream.next().await; + + let formatted = get_plan_string(&plan).join("\n"); + assert_contains!(&formatted, "probe_keys="); + assert_not_contains!(&formatted, "probe_keys=0"); + Ok(()) +} + #[test] fn multi_joins_after_alias() -> Result<()> { let left = parquet_exec();