diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index bec471bfe454..dd92a4465e97 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -725,11 +725,43 @@ 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 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'`. 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/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/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/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index fd847763124a..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}; @@ -562,7 +563,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)]", " ".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); @@ -586,8 +605,19 @@ fn multi_hash_joins() -> Result<()> { &top_join_on, &join_type, ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]"); + let 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 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" + ) + }; let expected = match join_type { // Should include 3 RepartitionExecs @@ -647,11 +677,21 @@ 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::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"), }; let expected = match join_type { @@ -701,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(); @@ -735,9 +802,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", @@ -762,9 +829,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", @@ -814,11 +881,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", @@ -859,7 +926,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 +979,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 +1100,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 +1110,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", @@ -1165,8 +1232,17 @@ fn reorder_join_keys_to_left_input() -> Result<()> { &top_join_on, &join_type, ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type); + let 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::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)?; @@ -1174,7 +1250,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", @@ -1182,7 +1258,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", @@ -1299,8 +1375,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)]", &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)?; @@ -1308,7 +1393,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", @@ -1316,7 +1401,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/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index d10459ce86ae..2443e8c52252 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -195,12 +195,12 @@ 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", ]; 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", ]; diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index 1a04753966a2..2747b83f743d 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; @@ -33,15 +32,17 @@ 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; 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, }; @@ -49,6 +50,7 @@ use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, coalesce_batches::CoalesceBatchesExec, filter::FilterExec, + joins::{utils::JoinFilter, HashJoinExec, PartitionMode}, repartition::RepartitionExec, sorts::sort::SortExec, ExecutionPlan, @@ -56,6 +58,9 @@ 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; @@ -73,7 +78,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 @@ -108,11 +113,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= ", ); } @@ -135,11 +140,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= " ); @@ -154,7 +159,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 @@ -164,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"]), @@ -244,8 +246,8 @@ 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] - - 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 + - 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=DynamicFilterPhysicalExpr [ true ] AND DynamicFilterPhysicalExpr [ true ] " ); @@ -266,12 +268,15 @@ 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] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - 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, 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 ] " ); + 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. @@ -280,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"]), @@ -366,13 +368,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)] - - 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=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 " @@ -412,15 +414,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: - - 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=a@0 = aa + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[d, e, f], file_type=test, pushdown_supported=true, predicate= " ); } @@ -441,7 +442,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 @@ -468,7 +469,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] @@ -491,7 +492,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] @@ -523,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 @@ -588,7 +589,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 @@ -615,7 +616,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 } @@ -634,12 +635,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= ", ); @@ -654,11 +655,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= ", ); } @@ -702,7 +703,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] @@ -731,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 ] " ); @@ -785,7 +786,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] @@ -821,10 +822,13 @@ 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" - - 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) ] " ); @@ -834,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"]), @@ -902,13 +903,13 @@ 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)] - - 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 + - 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)] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - 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 ] ", ); @@ -936,18 +937,23 @@ 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] - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true + - 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 ] " ); + + 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"]); } #[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 = @@ -962,8 +968,7 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { .with_support(true) .with_batches(t1_batches) .build(); - - // t2: larger table (will be probe side of inner join, build side of outer join) + // 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"]), @@ -980,10 +985,9 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { .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]) + ("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![ @@ -995,21 +999,13 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { .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())]; + // 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_join_on, + inner_on, None, &JoinType::Inner, None, @@ -1017,18 +1013,19 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { datafusion_common::NullEquality::NullEqualsNothing, ) .unwrap(), - ); + ) as Arc; - // Then create outer join: t1.a = t2.b (from inner join result) - let outer_join_on = vec![( + // 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_join.schema()).unwrap(), + col("b", &inner_schema).unwrap(), )]; let outer_join = Arc::new( HashJoinExec::try_new( t1_scan, - inner_join as Arc, - outer_join_on, + inner_join, + outer_on, None, &JoinType::Inner, None, @@ -1038,28 +1035,6 @@ async fn test_nested_hashjoin_dynamic_filter_pushdown() { .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)] - - 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 - 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)] - - 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; @@ -1082,60 +1057,168 @@ 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] - - 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=[(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_parent_filter_pushdown() { - use datafusion_common::JoinType; - use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; - - // Create build side with limited values +async fn test_hashjoin_dynamic_filter_pushdown_null_keys() { + // Build side containing only null join keys let build_batches = vec![record_batch!( - ("a", Utf8, ["aa", "ab"]), - ("b", Utf8, ["ba", "bb"]), - ("c", Float64, [1.0, 2.0]) + ("a", Utf8, [None::<&str>, None]), + ("b", Utf8, [None::<&str>, None]), + ("c", Float64, [Some(1.0), Some(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_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_side_schema)) + let build_scan = TestScanBuilder::new(Arc::clone(&build_schema)) .with_support(true) .with_batches(build_batches) .build(); - // Create probe side with more values + // Probe side with regular 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]) + ("a", Utf8, ["aa", "ab", "ac"]), + ("b", Utf8, ["ba", "bb", "bc"]), + ("e", Float64, [1.0, 2.0, 3.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_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_side_schema)) + let probe_scan = TestScanBuilder::new(Arc::clone(&probe_schema)) .with_support(true) .with_batches(probe_batches) .build(); - // Create HashJoinExec + 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 ]" + ); + + 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. + // 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"]); +} + +#[tokio::test] +async fn test_hashjoin_dynamic_filter_pushdown_high_cardinality() { + // 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_side_schema).unwrap(), - col("d", &probe_side_schema).unwrap(), + col("a", &build_schema).unwrap(), + col("a", &probe_schema).unwrap(), )]; - let join = Arc::new( + let plan = Arc::new( HashJoinExec::try_new( build_scan, probe_scan, @@ -1147,49 +1230,336 @@ async fn test_hashjoin_parent_filter_pushdown() { datafusion_common::NullEquality::NullEqualsNothing, ) .unwrap(), - ); + ) as Arc; - // 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(); + let mut config = ConfigOptions::default(); + config.execution.parquet.pushdown_filters = true; + config.optimizer.enable_dynamic_filter_pushdown = true; + 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"); - // 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(), + assert_contains!(&formatted, "probe_keys=0"); + + let join = plan.as_any().downcast_ref::().unwrap(); + assert_projection(&join.left, &["a", "x"]); + assert_projection(&join.right, &["a", "y"]); +} + +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"]), + ("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, + partition_mode, + datafusion_common::NullEquality::NullEqualsNothing, + ) + .unwrap(), + ) 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(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()]; + 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), + &join_type, + None, + PartitionMode::Partitioned, + datafusion_common::NullEquality::NullEqualsNothing, + ); + let err = res.expect_err("non-equi joins should be rejected"); + assert_not_contains!(err.to_string(), "non-equi"); +} - 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; +#[rstest(join_type, case::left(JoinType::Left), case::right(JoinType::Right))] +#[tokio::test] +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; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_dynamic_filter_location(&formatted, &join_type); + assert_contains!(&formatted, "probe_keys=0"); +} - // 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)] - - 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 - output: - Ok: - - FilterExec: a@0 = d@3 - - 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, 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 - " +#[rstest(join_type, case::left(JoinType::Left), case::right(JoinType::Right))] +#[tokio::test] +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; + 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_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=" + ); + assert_contains!(&formatted, "probe_keys=0"); +} + +#[rstest( + join_type, + case::left(JoinType::LeftSemi), + case::right(JoinType::RightSemi) +)] +#[tokio::test] +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; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + 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_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; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + 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_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; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + assert_dynamic_filter_location(&formatted, &join_type); + assert_contains!(&formatted, "probe_keys=0"); +} + +#[tokio::test] +async fn test_hashjoin_full_dynamic_filter_no_pushdown() { + 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; + let plan = FilterPushdown::new_post_optimization() + .optimize(plan, &config) + .unwrap(); + let formatted = format_plan_for_test(&plan); + 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( + join_type: JoinType, + left_support: bool, + right_support: bool, + partition_mode: PartitionMode, +) { + let plan = build_join_with_dynamic_filter( + join_type, + left_support, + right_support, + partition_mode, + ); + 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. @@ -1248,6 +1618,12 @@ 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 diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index acb2b808ef8f..3876f3d43d8c 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -143,7 +143,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 { @@ -195,18 +196,24 @@ 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() .map(|p| format!(", predicate={p}")) - .unwrap_or_default(); + .unwrap_or_else(|| ", predicate=".to_string()); 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/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/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 149c50557c3a..72eca494180a 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 " @@ -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 " 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", 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 - "### + " ); } 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; diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index 8ad7fa53c0e3..5cd20abb3e30 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -18,7 +18,6 @@ //! [`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::tree_node::Transformed; use datafusion_common::{NullEquality, Result}; @@ -54,8 +53,8 @@ impl OptimizerRule for FilterNullJoinKeys { if !join.on.is_empty() && join.null_equality == NullEquality::NullEqualsNothing => { - let (left_preserved, right_preserved) = - on_lr_is_preserved(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 62ee8c65bc1c..5a763d40a9ba 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; +// JoinType's preservation helpers determine output-filter safety and ON-clause pushdown. use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; @@ -155,50 +156,8 @@ pub struct PushDownFilter {} /// - 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 +/// 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 @@ -426,7 +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) = lr_is_preserved(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) @@ -463,7 +423,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 = 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-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index d4b3180a6fc6..b200dad4a0ab 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,9 @@ pub struct DynamicFilterPhysicalExpr { remapped_children: Option>>, /// 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. /// But this can have overhead in production, so it's only included in our tests. @@ -138,11 +144,13 @@ 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)), } } + #[inline] fn remap_children( children: &[Arc], remapped_children: Option<&Vec>>, @@ -175,15 +183,16 @@ 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(|_| { - datafusion_common::DataFusionError::Execution( - "Failed to acquire read lock for inner".to_string(), - ) + .map_err(|e| { + datafusion_common::DataFusionError::Execution(format!( + "Failed to acquire read lock for inner: {e}" + )) })? .expr, ); @@ -198,11 +207,19 @@ 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<()> { - let mut current = self.inner.write().map_err(|_| { - datafusion_common::DataFusionError::Execution( - "Failed to acquire write lock for inner".to_string(), - ) + /// + /// `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(|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 @@ -217,8 +234,25 @@ impl DynamicFilterPhysicalExpr { current.expr = new_expr; // Increment the generation to indicate that the expression has changed. current.generation += 1; + // 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 { + // See note in `update`; relaxed ordering is sufficient. + self.key_count.load(Ordering::Relaxed) + } } impl PhysicalExpr for DynamicFilterPhysicalExpr { @@ -242,6 +276,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 +468,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 +498,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 +527,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-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 383769173d7c..6195ea042bf3 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -94,6 +94,31 @@ use datafusion_physical_expr_common::physical_expr::fmt_sql; 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. +/// 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 = 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, + 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); @@ -160,6 +185,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 } @@ -366,7 +392,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 { @@ -388,7 +414,9 @@ impl HashJoinExec { 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"); + return plan_err!( + "HashJoinExec requires a non-empty ON clause; empty lists are unsupported" + ); } check_join_is_valid(&left_schema, &right_schema, &on)?; @@ -413,7 +441,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 +462,21 @@ 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(); + #[inline] + fn create_dynamic_filter( + on: &JoinOn, + join_type: JoinType, + ) -> 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 => return None, + }; // Initialize with a placeholder expression (true) that will be updated when the hash table is built - Arc::new(DynamicFilterPhysicalExpr::new(right_keys, lit(true))) + Some(Arc::new(DynamicFilterPhysicalExpr::new(keys, lit(true)))) } /// left (build) side which gets hashed @@ -686,11 +724,22 @@ 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}]") + let probe_filter_display = match &self.dynamic_filter { + Some(df) => { + let probe_side = dynamic_filter_side(self.join_type); + let keys = df.key_count(); + match df.current() { + Ok(current) if current != lit(true) => { + format!( + ", probe_filter=[{current}], probe_side={probe_side:?}, probe_keys={keys}" + ) + } + _ => { + format!(", probe_side={probe_side:?}, probe_keys={keys}") + } + } } - _ => "".to_string(), + None => "".to_string(), }; write!( f, @@ -700,7 +749,7 @@ impl DisplayAs for HashJoinExec { on, display_filter, display_projections, - dynamic_filter_display + probe_filter_display ) } DisplayFormatType::TreeRender => { @@ -713,16 +762,20 @@ 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}")?; - - if let Some(filter) = self.filter.as_ref() { - writeln!(f, "filter={filter}")?; + 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(()) } } @@ -804,8 +857,12 @@ impl ExecutionPlan for HashJoinExec { self.mode, self.null_equality, )?; - // Preserve the dynamic filter if it exists - new_join.dynamic_filter = Arc::clone(&self.dynamic_filter); + // 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)) } @@ -826,7 +883,9 @@ 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), + // 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), })) } @@ -886,7 +945,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(), )) })?, @@ -906,7 +966,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(), )) } @@ -1018,41 +1079,91 @@ impl ExecutionPlan for HashJoinExec { } } + #[inline] fn gather_filters_for_pushdown( &self, phase: FilterPushdownPhase, 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 = 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 + // 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 desc = crate::filter_pushdown::ChildFilterDescription::from_child( &parent_filters, - &self.children(), - )); - } + self.left(), + )?; + if !left_preserved { + // 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 + } 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 + || matches!(dynamic_target, JoinSide::Right) + { + let mut desc = crate::filter_pushdown::ChildFilterDescription::from_child( + &parent_filters, + self.right(), + )?; + if !right_preserved { + // 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, + self_filters: vec![], + } + }; - // Add dynamic filters in Post phase if enabled + // We only install dynamic filters after optimization to avoid planning scans + // 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 { - // Add actual dynamic filter to right side (probe side) - let dynamic_filter = - Arc::clone(&self.dynamic_filter) as Arc; - right_child = right_child.with_self_filter(dynamic_filter); + if let Some(df) = &self.dynamic_filter { + let dynamic_filter = Arc::clone(df) as Arc; + match dynamic_target { + 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() @@ -1066,18 +1177,10 @@ 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, - )); - } + // `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)) } } @@ -1252,7 +1355,7 @@ async fn collect_left_input( }) .unwrap_or_else(|| lit(true)); - dynamic_filter.update(combined_predicate)?; + dynamic_filter.update(combined_predicate, num_rows)?; } } @@ -1788,7 +1891,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, @@ -1925,6 +2028,62 @@ 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:?}"); + } + } + + #[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 = ( + 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(); + 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) {} diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index dc2a5640f40b..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,13 +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 { - if let Ok(current) = filter.current() { - if !current.eq(&lit(true)) { - write!(f, ", filter=[{current}]")?; - } - } - } + write_filter_info(f, &self.filter)?; if !self.common_sort_prefix.is_empty() { write!(f, ", sort_prefix=[")?; let mut first = true; @@ -1073,7 +1088,11 @@ 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)?; + write_filter_info(f, &self.filter)?; + 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..0c5ef4453ecb 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -404,9 +404,11 @@ 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 when the dynamic predicate is a tautology we still update + // `key_count` for observability. + filter.update(lit(true), self.heap.len())?; } Ok(()) @@ -610,6 +612,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. 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 54a2c5057e7c..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)│ ││ │ -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/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index c87abb972ea6..71a5d6d481e9 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 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`. 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 ad21bdac6d2d..607e362d4abd 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/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 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 diff --git a/dev/changelog/50.0.0.md b/dev/changelog/50.0.0.md new file mode 100644 index 000000000000..011ee66c406b --- /dev/null +++ b/dev/changelog/50.0.0.md @@ -0,0 +1,39 @@ + + +# Apache DataFusion 50.0.0 Changelog + +This release is under active development. + +- **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. 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 57ce6da117bb..880dfb15e942 100644 --- a/docs/source/library-user-guide/upgrading.md +++ b/docs/source/library-user-guide/upgrading.md @@ -48,6 +48,59 @@ 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. `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). 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 +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 | + +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 +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. +Plan effect: the `fact` scan receives `DynamicFilter{fact.id}`. + 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 f1dad3d360a0..6241949c6488 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 | 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 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. |