From 8f0c30fda713064424e9609bd49968009ce6136a Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Sun, 9 Jun 2024 20:36:07 +0300 Subject: [PATCH 01/28] Initial optimizer sanity checker. Only includes sort reqs, docs will be added. --- datafusion/core/src/physical_optimizer/mod.rs | 1 + .../core/src/physical_optimizer/optimizer.rs | 5 + .../src/physical_optimizer/sanity_checker.rs | 152 ++++++++++++++++++ 3 files changed, 158 insertions(+) create mode 100644 datafusion/core/src/physical_optimizer/sanity_checker.rs diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 7cc9a0fb75d4..2c1145f8829a 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -31,6 +31,7 @@ pub mod limited_distinct_aggregation; pub mod optimizer; pub mod output_requirements; pub mod pipeline_checker; +pub mod sanity_checker; mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index e3b60a0cca80..f63f755ec041 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -31,6 +31,7 @@ use crate::physical_optimizer::join_selection::JoinSelection; use crate::physical_optimizer::limited_distinct_aggregation::LimitedDistinctAggregation; use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_optimizer::pipeline_checker::PipelineChecker; +use crate::physical_optimizer::sanity_checker::SanityCheckPlan; use crate::physical_optimizer::topk_aggregation::TopKAggregation; use crate::{error::Result, physical_plan::ExecutionPlan}; @@ -124,6 +125,10 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), + // SanityChecker rule checks whether the order and the + // partition requirements of each execution plan is + // satisfied by its children or not. + Arc::new(SanityCheckPlan::new()), // The PipelineChecker rule will reject non-runnable query plans that use // pipeline-breaking operators on infinite input(s). The rule generates a // diagnostic error message when this happens. It makes no changes to the diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs new file mode 100644 index 000000000000..cca5bfaafbb5 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -0,0 +1,152 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +//http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use crate::error::Result; +use crate::physical_optimizer::PhysicalOptimizerRule; +use crate::physical_plan::ExecutionPlan; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::plan_err; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_physical_plan::ExecutionPlanProperties; + +#[derive(Default)] +pub struct SanityCheckPlan {} + +impl SanityCheckPlan { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for SanityCheckPlan { + fn optimize( + &self, + plan: Arc, + config: &ConfigOptions, + ) -> Result> { + plan.transform_down(|p| { + check_partition_requirements(check_sort_requirements(p)?.data) + }) + .data() + } + + fn name(&self) -> &str { + "SanityCheckPlan" + } + + fn schema_check(&self) -> bool { + true + } +} + +pub fn check_sort_requirements( + plan: Arc, +) -> Result>> { + let children = plan.children(); + let sort_reqs = plan.required_input_ordering(); + let children_len = children.len(); + // TODO: Use izip!. + for i in 0..children_len { + let child = &children[i]; + let child_sort_req = &sort_reqs[i]; + + // No requirement for the child + if child_sort_req.is_none() { + continue; + } + + let child_sort_req = child_sort_req.as_ref().unwrap(); + if !child + .equivalence_properties() + .ordering_satisfy_requirement(&child_sort_req) + { + return plan_err!( + "Child: {:?} does not satisfy parent order requirements", + child + ); + } + } + + Ok(Transformed::no(plan)) +} + +pub fn check_partition_requirements( + plan: Arc, +) -> Result>> { + Ok(Transformed::no(plan)) +} + +#[cfg(test)] +mod tests { + use super::*; + + use crate::physical_optimizer::test_utils::{ + bounded_window_exec, memory_exec, sort_exec, sort_expr_options + }; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use datafusion_common::Result; + + fn create_test_schema() -> Result { + let c9_column = Field::new("c9", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![c9_column])); + Ok(schema) + } + + #[tokio::test] + async fn test_bw_sort_requirement() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr_options( + "c9", + &source.schema(), + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source); + let bw = bounded_window_exec("c9", sort_exprs, sort); + let sanity_checker = SanityCheckPlan::new(); + let opts = ConfigOptions::default(); + assert_eq!(sanity_checker.optimize(bw, &opts).is_ok(), true); + Ok(()) + } + + #[tokio::test] + async fn test_bw_no_sort_requirement() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr_options( + "c9", + &source.schema(), + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let bw = bounded_window_exec("c9", sort_exprs, source); + let sanity_checker = SanityCheckPlan::new(); + let opts = ConfigOptions::default(); + assert_eq!(sanity_checker.optimize(bw, &opts).is_ok(), false); + Ok(()) + } +} From 25387026e5df540fa3ba2b0f5df8be6a3c15093d Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Tue, 11 Jun 2024 18:36:15 +0300 Subject: [PATCH 02/28] Add distro and pipeline friendly checks --- .../src/physical_optimizer/sanity_checker.rs | 58 ++++++++++--------- 1 file changed, 30 insertions(+), 28 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index cca5bfaafbb5..37b15a0db7e5 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use itertools::izip; use std::sync::Arc; use crate::error::Result; @@ -42,10 +43,7 @@ impl PhysicalOptimizerRule for SanityCheckPlan { plan: Arc, config: &ConfigOptions, ) -> Result> { - plan.transform_down(|p| { - check_partition_requirements(check_sort_requirements(p)?.data) - }) - .data() + plan.transform_down(|p| check_plan_sanity(p)).data() } fn name(&self) -> &str { @@ -57,29 +55,39 @@ impl PhysicalOptimizerRule for SanityCheckPlan { } } -pub fn check_sort_requirements( +/// Ensures that the plan is pipeline friendly and the order and +/// distribution requirements from its children are satisfied. +pub fn check_plan_sanity( plan: Arc, ) -> Result>> { - let children = plan.children(); - let sort_reqs = plan.required_input_ordering(); - let children_len = children.len(); - // TODO: Use izip!. - for i in 0..children_len { - let child = &children[i]; - let child_sort_req = &sort_reqs[i]; - - // No requirement for the child - if child_sort_req.is_none() { - continue; - } + if !plan.execution_mode().pipeline_friendly() { + return plan_err!("Plan {:?} is not pipeline friendly.", plan) + } + + for (child, child_sort_req, child_dist_req) in izip!( + plan.children().iter(), + plan.required_input_ordering().iter(), + plan.required_input_distribution().iter() + ) { + let child_eq_props = child.equivalence_properties(); + match child_sort_req { + None => (), + Some(child_sort_req) => { + if !child_eq_props.ordering_satisfy_requirement(&child_sort_req) { + return plan_err!( + "Child: {:?} does not satisfy parent order requirements", + child + ); + } + } + }; - let child_sort_req = child_sort_req.as_ref().unwrap(); if !child - .equivalence_properties() - .ordering_satisfy_requirement(&child_sort_req) + .output_partitioning() + .satisfy(&child_dist_req, child_eq_props) { return plan_err!( - "Child: {:?} does not satisfy parent order requirements", + "Child: {:?} does not satisfy parent distribution requirements", child ); } @@ -88,18 +96,12 @@ pub fn check_sort_requirements( Ok(Transformed::no(plan)) } -pub fn check_partition_requirements( - plan: Arc, -) -> Result>> { - Ok(Transformed::no(plan)) -} - #[cfg(test)] mod tests { use super::*; use crate::physical_optimizer::test_utils::{ - bounded_window_exec, memory_exec, sort_exec, sort_expr_options + bounded_window_exec, memory_exec, sort_exec, sort_expr_options, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; From aea95c66f7fe59826ea435597b2009a24bebf897 Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Wed, 12 Jun 2024 18:23:10 +0300 Subject: [PATCH 03/28] Also check the plans we create are correct. --- .../src/physical_optimizer/sanity_checker.rs | 49 +++++++++++++------ 1 file changed, 35 insertions(+), 14 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 37b15a0db7e5..2547c6818742 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -41,7 +41,7 @@ impl PhysicalOptimizerRule for SanityCheckPlan { fn optimize( &self, plan: Arc, - config: &ConfigOptions, + _config: &ConfigOptions, ) -> Result> { plan.transform_down(|p| check_plan_sanity(p)).data() } @@ -61,7 +61,7 @@ pub fn check_plan_sanity( plan: Arc, ) -> Result>> { if !plan.execution_mode().pipeline_friendly() { - return plan_err!("Plan {:?} is not pipeline friendly.", plan) + return plan_err!("Plan {:?} is not pipeline friendly.", plan); } for (child, child_sort_req, child_dist_req) in izip!( @@ -106,16 +106,32 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::Result; + use datafusion_physical_plan::displayable; - fn create_test_schema() -> Result { - let c9_column = Field::new("c9", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![c9_column])); - Ok(schema) + fn create_test_schema() -> SchemaRef { + Arc::new(Schema::new(vec![Field::new("c9", DataType::Int32, true)])) + } + + fn assert_sanity_check(plan: &Arc, is_sane: bool) { + let sanity_checker = SanityCheckPlan::new(); + let opts = ConfigOptions::default(); + assert_eq!( + sanity_checker.optimize(plan.clone(), &opts).is_ok(), + is_sane + ); + } + + /// Check if the plan we created is as expected by comparing the plan + /// formatted as a string. + fn assert_plan(plan: &dyn ExecutionPlan, expected_lines: Vec<&str>) { + let plan_str = displayable(plan).indent(true).to_string(); + let actual_lines: Vec<&str> = plan_str.trim().lines().collect(); + assert_eq!(actual_lines, expected_lines); } #[tokio::test] async fn test_bw_sort_requirement() -> Result<()> { - let schema = create_test_schema()?; + let schema = create_test_schema(); let source = memory_exec(&schema); let sort_exprs = vec![sort_expr_options( "c9", @@ -127,15 +143,18 @@ mod tests { )]; let sort = sort_exec(sort_exprs.clone(), source); let bw = bounded_window_exec("c9", sort_exprs, sort); - let sanity_checker = SanityCheckPlan::new(); - let opts = ConfigOptions::default(); - assert_eq!(sanity_checker.optimize(bw, &opts).is_ok(), true); + assert_plan(bw.as_ref(), vec![ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]" + ]); + assert_sanity_check(&bw, true); Ok(()) } #[tokio::test] async fn test_bw_no_sort_requirement() -> Result<()> { - let schema = create_test_schema()?; + let schema = create_test_schema(); let source = memory_exec(&schema); let sort_exprs = vec![sort_expr_options( "c9", @@ -146,9 +165,11 @@ mod tests { }, )]; let bw = bounded_window_exec("c9", sort_exprs, source); - let sanity_checker = SanityCheckPlan::new(); - let opts = ConfigOptions::default(); - assert_eq!(sanity_checker.optimize(bw, &opts).is_ok(), false); + assert_plan(bw.as_ref(), vec![ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " MemoryExec: partitions=1, partition_sizes=[0]" + ]); + assert_sanity_check(&bw, false); Ok(()) } } From 9aef599725b87998ff3adfe64ac9c7f49ac43cef Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Thu, 13 Jun 2024 19:58:23 +0300 Subject: [PATCH 04/28] Add distribution test cases using global limit exec. --- .../src/physical_optimizer/sanity_checker.rs | 66 ++++++++++++++++++- 1 file changed, 63 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 2547c6818742..4b06eea872b0 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -101,7 +101,8 @@ mod tests { use super::*; use crate::physical_optimizer::test_utils::{ - bounded_window_exec, memory_exec, sort_exec, sort_expr_options, + bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, + repartition_exec, sort_exec, sort_expr_options, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -130,7 +131,8 @@ mod tests { } #[tokio::test] - async fn test_bw_sort_requirement() -> Result<()> { + /// Tests that plan is valid when the sort requirements are satisfied. + async fn test_bounded_window_agg_sort_requirement() -> Result<()> { let schema = create_test_schema(); let source = memory_exec(&schema); let sort_exprs = vec![sort_expr_options( @@ -153,7 +155,8 @@ mod tests { } #[tokio::test] - async fn test_bw_no_sort_requirement() -> Result<()> { + /// Tests that plan is invalid when the sort requirements are not satisfied. + async fn test_bounded_window_agg_no_sort_requirement() -> Result<()> { let schema = create_test_schema(); let source = memory_exec(&schema); let sort_exprs = vec![sort_expr_options( @@ -172,4 +175,61 @@ mod tests { assert_sanity_check(&bw, false); Ok(()) } + + #[tokio::test] + /// Tests that plan is valid when a single partition requirement + /// is satisfied. + async fn test_global_limit_single_partition() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = global_limit_exec(source); + + assert_plan( + limit.as_ref(), + vec![ + "GlobalLimitExec: skip=0, fetch=100", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&limit, true); + Ok(()) + } + + #[tokio::test] + /// Tests that plan is invalid when a single partition requirement + /// is not satisfied. + async fn test_global_limit_multi_partition() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = global_limit_exec(repartition_exec(source)); + + assert_plan( + limit.as_ref(), + vec![ + "GlobalLimitExec: skip=0, fetch=100", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&limit, false); + Ok(()) + } + + #[tokio::test] + /// Tests that when a plan has no requirements it is valid. + async fn test_local_limit() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = local_limit_exec(source); + + assert_plan( + limit.as_ref(), + vec![ + "LocalLimitExec: fetch=100", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&limit, true); + Ok(()) + } } From 49aba4b0a541e6bb103f3aa4392589fc30298e5f Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Sat, 15 Jun 2024 13:40:55 +0300 Subject: [PATCH 05/28] Add test for multiple children using SortMergeJoinExec. --- .../src/physical_optimizer/sanity_checker.rs | 142 +++++++++++++++++- 1 file changed, 141 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 4b06eea872b0..4b6c797c8d47 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -102,17 +102,28 @@ mod tests { use crate::physical_optimizer::test_utils::{ bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, - repartition_exec, sort_exec, sort_expr_options, + repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::Result; + use datafusion_expr::JoinType; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::displayable; + use datafusion_physical_plan::repartition::RepartitionExec; fn create_test_schema() -> SchemaRef { Arc::new(Schema::new(vec![Field::new("c9", DataType::Int32, true)])) } + fn create_test_schema2() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ])) + } + fn assert_sanity_check(plan: &Arc, is_sane: bool) { let sanity_checker = SanityCheckPlan::new(); let opts = ConfigOptions::default(); @@ -232,4 +243,133 @@ mod tests { assert_sanity_check(&limit, true); Ok(()) } + + #[tokio::test] + /// Tests that plan is valid when the sort requirements are satisfied. + async fn test_sort_merge_join_satisfied() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let source2 = memory_exec(&schema2); + let sort_opts = SortOptions::default(); + let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; + let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; + let left = sort_exec(sort_exprs1, source1); + let right = sort_exec(sort_exprs2, source2); + let left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + let right = Arc::new(RepartitionExec::try_new( + right, + Partitioning::Hash(vec![right_jcol.clone()], 10), + )?); + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", + " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&smj, true); + Ok(()) + } + + #[tokio::test] + /// Tests that plan is valid when the sort requirements are satisfied. + async fn test_sort_merge_join_order_missing() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let right = memory_exec(&schema2); + let sort_exprs1 = vec![sort_expr_options( + "c9", + &source1.schema(), + SortOptions::default(), + )]; + let left = sort_exec(sort_exprs1, source1); + // Missing sort of the right child here.. + let left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + let right = Arc::new(RepartitionExec::try_new( + right, + Partitioning::Hash(vec![right_jcol.clone()], 10), + )?); + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", + " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&smj, false); + Ok(()) + } + + #[tokio::test] + /// Tests that plan is valid when the sort requirements are satisfied. + async fn test_sort_merge_join_dist_missing() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let source2 = memory_exec(&schema2); + let sort_opts = SortOptions::default(); + let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; + let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; + let left = sort_exec(sort_exprs1, source1); + let right = sort_exec(sort_exprs2, source2); + let left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + // Missing hash partitioning here.. + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", + " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&smj, false); + Ok(()) + } } From d89b1c91c9d8e8571dd2757393ed6fe7e0619cdf Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Sun, 16 Jun 2024 17:22:07 +0300 Subject: [PATCH 06/28] Move PipelineChecker to SanityCheckPlan --- datafusion/core/src/physical_optimizer/mod.rs | 1 - .../core/src/physical_optimizer/optimizer.rs | 17 +- .../physical_optimizer/pipeline_checker.rs | 334 ------------------ .../src/physical_optimizer/sanity_checker.rs | 284 ++++++++++++++- 4 files changed, 286 insertions(+), 350 deletions(-) delete mode 100644 datafusion/core/src/physical_optimizer/pipeline_checker.rs diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 2c1145f8829a..f4b93f53ad55 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -30,7 +30,6 @@ pub mod join_selection; pub mod limited_distinct_aggregation; pub mod optimizer; pub mod output_requirements; -pub mod pipeline_checker; pub mod sanity_checker; mod projection_pushdown; pub mod pruning; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index f63f755ec041..5fd7d848d2a2 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -30,7 +30,6 @@ use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::join_selection::JoinSelection; use crate::physical_optimizer::limited_distinct_aggregation::LimitedDistinctAggregation; use crate::physical_optimizer::output_requirements::OutputRequirements; -use crate::physical_optimizer::pipeline_checker::PipelineChecker; use crate::physical_optimizer::sanity_checker::SanityCheckPlan; use crate::physical_optimizer::topk_aggregation::TopKAggregation; use crate::{error::Result, physical_plan::ExecutionPlan}; @@ -125,15 +124,15 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), - // SanityChecker rule checks whether the order and the - // partition requirements of each execution plan is - // satisfied by its children or not. + // The SanityCheckPlan will reject non-runnable query + // plans that use pipeline-breaking operators on infinite + // input(s) as well as checks whether the order and + // distribution requirements of each children of a plan + // are satisfied. The rule generates a diagnostic error + // message when this happens. It makes no changes to the + // given query plan; i.e. it only acts as a final + // gatekeeping rule. Arc::new(SanityCheckPlan::new()), - // The PipelineChecker rule will reject non-runnable query plans that use - // pipeline-breaking operators on infinite input(s). The rule generates a - // diagnostic error message when this happens. It makes no changes to the - // given query plan; i.e. it only acts as a final gatekeeping rule. - Arc::new(PipelineChecker::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs deleted file mode 100644 index 5c6a0ab8ea7f..000000000000 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ /dev/null @@ -1,334 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -//http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! The [PipelineChecker] rule ensures that a given plan can accommodate its -//! infinite sources, if there are any. It will reject non-runnable query plans -//! that use pipeline-breaking operators on infinite input(s). - -use std::sync::Arc; - -use crate::config::ConfigOptions; -use crate::error::Result; -use crate::physical_optimizer::PhysicalOptimizerRule; -use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; - -use datafusion_common::config::OptimizerOptions; -use datafusion_common::plan_err; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; -use datafusion_physical_plan::joins::SymmetricHashJoinExec; - -/// The PipelineChecker rule rejects non-runnable query plans that use -/// pipeline-breaking operators on infinite input(s). -#[derive(Default)] -pub struct PipelineChecker {} - -impl PipelineChecker { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl PhysicalOptimizerRule for PipelineChecker { - fn optimize( - &self, - plan: Arc, - config: &ConfigOptions, - ) -> Result> { - plan.transform_up(|p| check_finiteness_requirements(p, &config.optimizer)) - .data() - } - - fn name(&self) -> &str { - "PipelineChecker" - } - - fn schema_check(&self) -> bool { - true - } -} - -/// This function propagates finiteness information and rejects any plan with -/// pipeline-breaking operators acting on infinite inputs. -pub fn check_finiteness_requirements( - input: Arc, - optimizer_options: &OptimizerOptions, -) -> Result>> { - if let Some(exec) = input.as_any().downcast_ref::() { - if !(optimizer_options.allow_symmetric_joins_without_pruning - || (exec.check_if_order_information_available()? && is_prunable(exec))) - { - return plan_err!("Join operation cannot operate on a non-prunable stream without enabling \ - the 'allow_symmetric_joins_without_pruning' configuration flag"); - } - } - if !input.execution_mode().pipeline_friendly() { - plan_err!( - "Cannot execute pipeline breaking queries, operator: {:?}", - input - ) - } else { - Ok(Transformed::no(input)) - } -} - -/// This function returns whether a given symmetric hash join is amenable to -/// data pruning. For this to be possible, it needs to have a filter where -/// all involved [`PhysicalExpr`]s, [`Operator`]s and data types support -/// interval calculations. -/// -/// [`PhysicalExpr`]: crate::physical_plan::PhysicalExpr -/// [`Operator`]: datafusion_expr::Operator -fn is_prunable(join: &SymmetricHashJoinExec) -> bool { - join.filter().map_or(false, |filter| { - check_support(filter.expression(), &join.schema()) - && filter - .schema() - .fields() - .iter() - .all(|f| is_datatype_supported(f.data_type())) - }) -} - -#[cfg(test)] -mod sql_tests { - use super::*; - use crate::physical_optimizer::test_utils::{ - BinaryTestCase, QueryCase, SourceType, UnaryTestCase, - }; - - #[tokio::test] - async fn test_hash_left_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: false, - }; - - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 LEFT JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) - } - - #[tokio::test] - async fn test_hash_right_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 RIGHT JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) - } - - #[tokio::test] - async fn test_hash_inner_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: false, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "Join Error".to_string(), - }; - - case.run().await?; - Ok(()) - } - - #[tokio::test] - async fn test_hash_full_outer_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 FULL JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) - } - - #[tokio::test] - async fn test_aggregate() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT c1, MIN(c4) FROM test GROUP BY c1".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: AggregateExec".to_string(), - }; - - case.run().await?; - Ok(()) - } - - #[tokio::test] - async fn test_window_agg_hash_partition() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT - c9, - SUM(c9) OVER(PARTITION BY c1 ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 - FROM test - LIMIT 5".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: SortExec".to_string() - }; - - case.run().await?; - Ok(()) - } - - #[tokio::test] - async fn test_window_agg_single_partition() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT - c9, - SUM(c9) OVER(ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 - FROM test".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: SortExec".to_string() - }; - case.run().await?; - Ok(()) - } - - #[tokio::test] - async fn test_hash_cross_join() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: true, - }; - let test4 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 CROSS JOIN right as t2".to_string(), - cases: vec![ - Arc::new(test1), - Arc::new(test2), - Arc::new(test3), - Arc::new(test4), - ], - error_operator: "operator: CrossJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) - } - - #[tokio::test] - async fn test_analyzer() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: false, - }; - let case = QueryCase { - sql: "EXPLAIN ANALYZE SELECT * FROM test".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "Analyze Error".to_string(), - }; - - case.run().await?; - Ok(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 4b6c797c8d47..6b747a49f75d 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -15,6 +15,10 @@ // specific language governing permissions and limitations // under the License. +//! The [PipelineChecker] rule ensures that a given plan can accommodate its +//! infinite sources, if there are any. It will reject non-runnable query plans +//! that use pipeline-breaking operators on infinite input(s). + use itertools::izip; use std::sync::Arc; @@ -22,11 +26,15 @@ use crate::error::Result; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::ExecutionPlan; -use datafusion_common::config::ConfigOptions; +use datafusion_common::config::{ConfigOptions, OptimizerOptions}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; +use datafusion_physical_plan::joins::SymmetricHashJoinExec; use datafusion_physical_plan::ExecutionPlanProperties; +/// The PipelineChecker rule rejects non-runnable query plans that use +/// pipeline-breaking operators on infinite input(s). #[derive(Default)] pub struct SanityCheckPlan {} @@ -41,9 +49,10 @@ impl PhysicalOptimizerRule for SanityCheckPlan { fn optimize( &self, plan: Arc, - _config: &ConfigOptions, + config: &ConfigOptions, ) -> Result> { - plan.transform_down(|p| check_plan_sanity(p)).data() + plan.transform_down(|p| check_plan_sanity(p, &config.optimizer)) + .data() } fn name(&self) -> &str { @@ -55,14 +64,55 @@ impl PhysicalOptimizerRule for SanityCheckPlan { } } +/// This function propagates finiteness information and rejects any plan with +/// pipeline-breaking operators acting on infinite inputs. +pub fn check_finiteness_requirements( + input: Arc, + optimizer_options: &OptimizerOptions, +) -> Result>> { + if let Some(exec) = input.as_any().downcast_ref::() { + if !(optimizer_options.allow_symmetric_joins_without_pruning + || (exec.check_if_order_information_available()? && is_prunable(exec))) + { + return plan_err!("Join operation cannot operate on a non-prunable stream without enabling \ + the 'allow_symmetric_joins_without_pruning' configuration flag"); + } + } + if !input.execution_mode().pipeline_friendly() { + plan_err!( + "Cannot execute pipeline breaking queries, operator: {:?}", + input + ) + } else { + Ok(Transformed::no(input)) + } +} + +/// This function returns whether a given symmetric hash join is amenable to +/// data pruning. For this to be possible, it needs to have a filter where +/// all involved [`PhysicalExpr`]s, [`Operator`]s and data types support +/// interval calculations. +/// +/// [`PhysicalExpr`]: crate::physical_plan::PhysicalExpr +/// [`Operator`]: datafusion_expr::Operator +fn is_prunable(join: &SymmetricHashJoinExec) -> bool { + join.filter().map_or(false, |filter| { + check_support(filter.expression(), &join.schema()) + && filter + .schema() + .fields() + .iter() + .all(|f| is_datatype_supported(f.data_type())) + }) +} + /// Ensures that the plan is pipeline friendly and the order and /// distribution requirements from its children are satisfied. pub fn check_plan_sanity( plan: Arc, + optimizer_options: &OptimizerOptions, ) -> Result>> { - if !plan.execution_mode().pipeline_friendly() { - return plan_err!("Plan {:?} is not pipeline friendly.", plan); - } + check_finiteness_requirements(plan.clone(), optimizer_options)?; for (child, child_sort_req, child_dist_req) in izip!( plan.children().iter(), @@ -103,7 +153,9 @@ mod tests { use crate::physical_optimizer::test_utils::{ bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, + BinaryTestCase, QueryCase, SourceType, UnaryTestCase, }; + use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::Result; @@ -141,6 +193,226 @@ mod tests { assert_eq!(actual_lines, expected_lines); } + #[tokio::test] + async fn test_hash_left_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: false, + }; + + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: true, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 LEFT JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) + } + + #[tokio::test] + async fn test_hash_right_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 RIGHT JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) + } + + #[tokio::test] + async fn test_hash_inner_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: false, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "Join Error".to_string(), + }; + + case.run().await?; + Ok(()) + } + + #[tokio::test] + async fn test_hash_full_outer_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: true, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 FULL JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) + } + + #[tokio::test] + async fn test_aggregate() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT c1, MIN(c4) FROM test GROUP BY c1".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: AggregateExec".to_string(), + }; + + case.run().await?; + Ok(()) + } + + #[tokio::test] + async fn test_window_agg_hash_partition() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT + c9, + SUM(c9) OVER(PARTITION BY c1 ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 + FROM test + LIMIT 5".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: SortExec".to_string() + }; + + case.run().await?; + Ok(()) + } + + #[tokio::test] + async fn test_window_agg_single_partition() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT + c9, + SUM(c9) OVER(ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 + FROM test".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: SortExec".to_string() + }; + case.run().await?; + Ok(()) + } + + #[tokio::test] + async fn test_hash_cross_join() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Unbounded), + expect_fail: true, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: true, + }; + let test4 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 CROSS JOIN right as t2".to_string(), + cases: vec![ + Arc::new(test1), + Arc::new(test2), + Arc::new(test3), + Arc::new(test4), + ], + error_operator: "operator: CrossJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) + } + + #[tokio::test] + async fn test_analyzer() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: false, + }; + let case = QueryCase { + sql: "EXPLAIN ANALYZE SELECT * FROM test".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "Analyze Error".to_string(), + }; + + case.run().await?; + Ok(()) + } + #[tokio::test] /// Tests that plan is valid when the sort requirements are satisfied. async fn test_bounded_window_agg_sort_requirement() -> Result<()> { From defebb0f33fcaebede25a66ad05f65683bdb7e4a Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Sun, 16 Jun 2024 19:01:43 +0300 Subject: [PATCH 07/28] Fix some tests and add docs --- .../src/physical_optimizer/sanity_checker.rs | 18 ++++++++++++------ datafusion/core/tests/sql/joins.rs | 2 +- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 6b747a49f75d..6d0b377a57d9 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -15,9 +15,11 @@ // specific language governing permissions and limitations // under the License. -//! The [PipelineChecker] rule ensures that a given plan can accommodate its -//! infinite sources, if there are any. It will reject non-runnable query plans -//! that use pipeline-breaking operators on infinite input(s). +//! The [SanityCheckPlan] rule ensures that a given plan can +//! accommodate its infinite sources, if there are any. It will reject +//! non-runnable query plans that use pipeline-breaking operators on +//! infinite input(s). In addition, it will check if all order and +//! distribution requirements of a plan are satisfied by its children. use itertools::izip; use std::sync::Arc; @@ -33,8 +35,12 @@ use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supp use datafusion_physical_plan::joins::SymmetricHashJoinExec; use datafusion_physical_plan::ExecutionPlanProperties; -/// The PipelineChecker rule rejects non-runnable query plans that use -/// pipeline-breaking operators on infinite input(s). +/// The SanityCheckPlan rule rejects the following query plans: +/// i) Plans that use pipeline-breaking operators on infinite input(s), +/// these queries cannot be ran. +/// ii) Plans in which their order and distribution requirements are not +/// satisfied by its children, these queries will most likely yield +/// incorrect results. #[derive(Default)] pub struct SanityCheckPlan {} @@ -51,7 +57,7 @@ impl PhysicalOptimizerRule for SanityCheckPlan { plan: Arc, config: &ConfigOptions, ) -> Result> { - plan.transform_down(|p| check_plan_sanity(p, &config.optimizer)) + plan.transform_up(|p| check_plan_sanity(p, &config.optimizer)) .data() } diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index f7d5205db0d3..6cdb7a111dd8 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -226,7 +226,7 @@ async fn join_change_in_planner_without_sort_not_allowed() -> Result<()> { match df.create_physical_plan().await { Ok(_) => panic!("Expecting error."), Err(e) => { - assert_eq!(e.strip_backtrace(), "PipelineChecker\ncaused by\nError during planning: Join operation cannot operate on a non-prunable stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag") + assert_eq!(e.strip_backtrace(), "SanityCheckPlan\ncaused by\nError during planning: Join operation cannot operate on a non-prunable stream without enabling the 'allow_symmetric_joins_without_pruning' configuration flag") } } Ok(()) From b392cf6475591c71df971206b322a16b94d2c6f8 Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Mon, 17 Jun 2024 14:42:00 +0300 Subject: [PATCH 08/28] Add some test docs and fix clippy diagnostics. --- datafusion/core/src/physical_optimizer/mod.rs | 2 +- .../src/physical_optimizer/sanity_checker.rs | 21 +++++++++++-------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index f4b93f53ad55..70f34a9c7090 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -30,10 +30,10 @@ pub mod join_selection; pub mod limited_distinct_aggregation; pub mod optimizer; pub mod output_requirements; -pub mod sanity_checker; mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; +pub mod sanity_checker; mod sort_pushdown; pub mod topk_aggregation; mod update_aggr_exprs; diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 6d0b377a57d9..a08da7baf8da 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -129,7 +129,7 @@ pub fn check_plan_sanity( match child_sort_req { None => (), Some(child_sort_req) => { - if !child_eq_props.ordering_satisfy_requirement(&child_sort_req) { + if !child_eq_props.ordering_satisfy_requirement(child_sort_req) { return plan_err!( "Child: {:?} does not satisfy parent order requirements", child @@ -140,7 +140,7 @@ pub fn check_plan_sanity( if !child .output_partitioning() - .satisfy(&child_dist_req, child_eq_props) + .satisfy(child_dist_req, child_eq_props) { return plan_err!( "Child: {:?} does not satisfy parent distribution requirements", @@ -182,6 +182,7 @@ mod tests { ])) } + /// Check if sanity checker should accept or reject plans. fn assert_sanity_check(plan: &Arc, is_sane: bool) { let sanity_checker = SanityCheckPlan::new(); let opts = ConfigOptions::default(); @@ -466,7 +467,7 @@ mod tests { } #[tokio::test] - /// Tests that plan is valid when a single partition requirement + /// A valid when a single partition requirement /// is satisfied. async fn test_global_limit_single_partition() -> Result<()> { let schema = create_test_schema(); @@ -485,7 +486,7 @@ mod tests { } #[tokio::test] - /// Tests that plan is invalid when a single partition requirement + /// An invalid plan when a single partition requirement /// is not satisfied. async fn test_global_limit_multi_partition() -> Result<()> { let schema = create_test_schema(); @@ -505,7 +506,7 @@ mod tests { } #[tokio::test] - /// Tests that when a plan has no requirements it is valid. + /// A plan with no requirements should satisfy. async fn test_local_limit() -> Result<()> { let schema = create_test_schema(); let source = memory_exec(&schema); @@ -523,7 +524,7 @@ mod tests { } #[tokio::test] - /// Tests that plan is valid when the sort requirements are satisfied. + /// Valid plan with multiple children satisfy both order and distribution. async fn test_sort_merge_join_satisfied() -> Result<()> { let schema1 = create_test_schema(); let schema2 = create_test_schema2(); @@ -567,7 +568,8 @@ mod tests { } #[tokio::test] - /// Tests that plan is valid when the sort requirements are satisfied. + /// Invalid case when the order is not satisfied by the 2nd + /// child. async fn test_sort_merge_join_order_missing() -> Result<()> { let schema1 = create_test_schema(); let schema2 = create_test_schema2(); @@ -612,7 +614,8 @@ mod tests { } #[tokio::test] - /// Tests that plan is valid when the sort requirements are satisfied. + /// Invalid case when the distribution is not satisfied by the 2nd + /// child. async fn test_sort_merge_join_dist_missing() -> Result<()> { let schema1 = create_test_schema(); let schema2 = create_test_schema2(); @@ -630,7 +633,7 @@ mod tests { Partitioning::Hash(vec![left_jcol.clone()], 10), )?); - // Missing hash partitioning here.. + // Missing hash partitioning on right child. let join_on = vec![(left_jcol as _, right_jcol as _)]; let join_ty = JoinType::Inner; From 5bdec622138c1fdbf4095a9df87ff18a5024592a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 20 Jun 2024 10:35:43 +0300 Subject: [PATCH 09/28] Fix some failing tests --- .../src/physical_optimizer/sanity_checker.rs | 9 ++- datafusion/core/tests/fifo/mod.rs | 58 +++++++++---------- datafusion/core/tests/tpcds_planning.rs | 5 +- datafusion/physical-expr/src/partitioning.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 2 +- 5 files changed, 44 insertions(+), 34 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index a08da7baf8da..52a8581d8ced 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -626,6 +626,10 @@ mod tests { let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; let left = sort_exec(sort_exprs1, source1); let right = sort_exec(sort_exprs2, source2); + let right = Arc::new(RepartitionExec::try_new( + right, + Partitioning::RoundRobinBatch(10), + )?); let left_jcol = col("c9", &left.schema()).unwrap(); let right_jcol = col("a", &right.schema()).unwrap(); let left = Arc::new(RepartitionExec::try_new( @@ -646,8 +650,9 @@ mod tests { " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", ], ); assert_sanity_check(&smj, false); diff --git a/datafusion/core/tests/fifo/mod.rs b/datafusion/core/tests/fifo/mod.rs index a63240d03d94..d03895664e91 100644 --- a/datafusion/core/tests/fifo/mod.rs +++ b/datafusion/core/tests/fifo/mod.rs @@ -217,17 +217,6 @@ mod unix_test { .set_bool("datafusion.execution.coalesce_batches", false) .with_target_partitions(1); let ctx = SessionContext::new_with_config(config); - // Tasks - let mut tasks: Vec> = vec![]; - - // Join filter - let a1_iter = 0..TEST_DATA_SIZE; - // Join key - let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); - let lines = a1_iter - .zip(a2_iter) - .map(|(a1, a2)| format!("{a1},{a2}\n")) - .collect::>(); // Create a new temporary FIFO file let tmp_dir = TempDir::new()?; @@ -238,22 +227,6 @@ mod unix_test { // Create a mutex for tracking if the right input source is waiting for data. let waiting = Arc::new(AtomicBool::new(true)); - // Create writing threads for the left and right FIFO files - tasks.push(create_writing_thread( - left_fifo.clone(), - "a1,a2\n".to_owned(), - lines.clone(), - waiting.clone(), - TEST_BATCH_SIZE, - )); - tasks.push(create_writing_thread( - right_fifo.clone(), - "a1,a2\n".to_owned(), - lines.clone(), - waiting.clone(), - TEST_BATCH_SIZE, - )); - // Create schema let schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::UInt32, false), @@ -264,10 +237,10 @@ mod unix_test { let order = vec![vec![datafusion_expr::col("a1").sort(true, false)]]; // Set unbounded sorted files read configuration - let provider = fifo_table(schema.clone(), left_fifo, order.clone()); + let provider = fifo_table(schema.clone(), left_fifo.clone(), order.clone()); ctx.register_table("left", provider)?; - let provider = fifo_table(schema.clone(), right_fifo, order); + let provider = fifo_table(schema.clone(), right_fifo.clone(), order); ctx.register_table("right", provider)?; // Execute the query, with no matching rows. (since key is modulus 10) @@ -287,6 +260,33 @@ mod unix_test { .await?; let mut stream = df.execute_stream().await?; let mut operations = vec![]; + + // Tasks + let mut tasks: Vec> = vec![]; + + // Join filter + let a1_iter = 0..TEST_DATA_SIZE; + // Join key + let a2_iter = (0..TEST_DATA_SIZE).map(|x| x % 10); + let lines = a1_iter + .zip(a2_iter) + .map(|(a1, a2)| format!("{a1},{a2}\n")) + .collect::>(); + // Create writing threads for the left and right FIFO files + tasks.push(create_writing_thread( + left_fifo.clone(), + "a1,a2\n".to_owned(), + lines.clone(), + waiting.clone(), + TEST_BATCH_SIZE, + )); + tasks.push(create_writing_thread( + right_fifo.clone(), + "a1,a2\n".to_owned(), + lines.clone(), + waiting.clone(), + TEST_BATCH_SIZE, + )); // Partial. while let Some(Ok(batch)) = stream.next().await { waiting.store(false, Ordering::SeqCst); diff --git a/datafusion/core/tests/tpcds_planning.rs b/datafusion/core/tests/tpcds_planning.rs index 44fb0afff319..b99bc2680044 100644 --- a/datafusion/core/tests/tpcds_planning.rs +++ b/datafusion/core/tests/tpcds_planning.rs @@ -1044,7 +1044,10 @@ async fn regression_test(query_no: u8, create_physical: bool) -> Result<()> { for table in &tables { ctx.register_table( table.name.as_str(), - Arc::new(MemTable::try_new(Arc::new(table.schema.clone()), vec![])?), + Arc::new(MemTable::try_new( + Arc::new(table.schema.clone()), + vec![vec![]], + )?), )?; } diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index fcb3278b6022..273c77fb1d5e 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -152,6 +152,8 @@ impl Partitioning { match required { Distribution::UnspecifiedDistribution => true, Distribution::SinglePartition if self.partition_count() == 1 => true, + // When partition count is 1, hash requirement is satisfied. + Distribution::HashPartitioned(_) if self.partition_count() == 1 => true, Distribution::HashPartitioned(required_exprs) => { match self { // Here we do not check the partition count for hash partitioning and assumes the partition count @@ -290,7 +292,7 @@ mod tests { assert_eq!(result, (true, false, false, false, false)) } Distribution::HashPartitioned(_) => { - assert_eq!(result, (false, false, false, true, false)) + assert_eq!(result, (true, false, false, true, false)) } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2bb95852ff43..35ec3671e623 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -675,7 +675,7 @@ impl ExecutionPlan for AggregateExec { vec![Distribution::UnspecifiedDistribution] } AggregateMode::FinalPartitioned | AggregateMode::SinglePartitioned => { - vec![Distribution::HashPartitioned(self.output_group_expr())] + vec![Distribution::HashPartitioned(self.group_by.input_exprs())] } AggregateMode::Final | AggregateMode::Single => { vec![Distribution::SinglePartition] From 7148c0760d554d9eda9970dd9244cbc7d4e05f69 Mon Sep 17 00:00:00 2001 From: Erman Yafay Date: Fri, 21 Jun 2024 10:01:59 +0300 Subject: [PATCH 10/28] Replace PipelineChecker with SanityChecker in .slt files. --- datafusion/sqllogictest/test_files/explain.slt | 6 +++--- datafusion/sqllogictest/test_files/window.slt | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 3c5f8c7f7ad6..9636683fce7f 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -256,7 +256,7 @@ physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after PipelineChecker SAME TEXT AS ABOVE +physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -315,7 +315,7 @@ physical_plan after OutputRequirements 02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after PipelineChecker SAME TEXT AS ABOVE +physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -352,7 +352,7 @@ physical_plan after OutputRequirements 02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after PipelineChecker SAME TEXT AS ABOVE +physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan 01)GlobalLimitExec: skip=0, fetch=10 02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 77b839f3f77a..ded20f513364 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3190,8 +3190,8 @@ SELECT a, d, rn1, rank1 FROM (SELECT a, d, # this is a negative test for asserting that ROW_NUMBER is not # added to the ordering equivalence when it contains partition by. # physical plan should contain SortExec. Since source is unbounded -# pipeline checker should raise error, when plan contains SortExec. -statement error DataFusion error: PipelineChecker +# sanity checker should raise error, when plan contains SortExec. +statement error DataFusion error: SanityCheckPlan SELECT a, d, rn1 FROM (SELECT a, d, ROW_NUMBER() OVER(PARTITION BY d ORDER BY a ASC) as rn1 FROM annotated_data_infinite2 From 1e888c40afb98ff9b6cfa3b35e966f7088bead33 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jun 2024 15:52:50 +0300 Subject: [PATCH 11/28] Initial commit --- .../src/physical_optimizer/enforce_sorting.rs | 22 +++- .../physical_optimizer/output_requirements.rs | 2 +- .../src/physical_optimizer/sort_pushdown.rs | 31 +++++- .../physical-expr/src/equivalence/mod.rs | 2 +- .../physical-expr/src/equivalence/ordering.rs | 15 +++ .../src/equivalence/properties.rs | 105 +++++++++++++----- datafusion/physical-expr/src/lib.rs | 1 + .../physical-plan/src/coalesce_partitions.rs | 3 +- datafusion/physical-plan/src/filter.rs | 22 +++- .../physical-plan/src/repartition/mod.rs | 6 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- datafusion/physical-plan/src/union.rs | 96 ++++++++++------ datafusion/physical-plan/src/windows/mod.rs | 12 +- .../sqllogictest/test_files/aggregate.slt | 2 +- datafusion/sqllogictest/test_files/joins.slt | 56 +++++----- datafusion/sqllogictest/test_files/window.slt | 2 +- 16 files changed, 277 insertions(+), 102 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 24306647c686..7d6f34fcd17e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -64,7 +64,7 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; -use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::{displayable, ExecutionPlanProperties}; use itertools::izip; @@ -147,6 +147,12 @@ fn update_coalesce_ctx_children( }; } +fn print_plan(plan: &Arc) { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + // println!("{:#?}", actual); +} + /// The boolean flag `repartition_sorts` defined in the config indicates /// whether we elect to transform [`CoalescePartitionsExec`] + [`SortExec`] cascades /// into [`SortExec`] + [`SortPreservingMergeExec`] cascades, which enables us to @@ -157,10 +163,14 @@ impl PhysicalOptimizerRule for EnforceSorting { plan: Arc, config: &ConfigOptions, ) -> Result> { + println!("start enforce req"); + print_plan(&plan); let plan_requirements = PlanWithCorrespondingSort::new_default(plan); // Execute a bottom-up traversal to enforce sorting requirements, // remove unnecessary sorts, and optimize sort-sensitive operators: let adjusted = plan_requirements.transform_up(ensure_sorting)?.data; + println!("AFTER enforce req"); + print_plan(&adjusted.plan); let new_plan = if config.optimizer.repartition_sorts { let plan_with_coalesce_partitions = PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan); @@ -171,6 +181,8 @@ impl PhysicalOptimizerRule for EnforceSorting { } else { adjusted.plan }; + println!("AFTER parallelize"); + print_plan(&new_plan); let plan_with_pipeline_fixer = OrderPreservationContext::new_default(new_plan); let updated_plan = plan_with_pipeline_fixer @@ -183,13 +195,16 @@ impl PhysicalOptimizerRule for EnforceSorting { ) }) .data()?; - + println!("AFTER replace order preserving variant"); + print_plan(&updated_plan.plan); // Execute a top-down traversal to exploit sort push-down opportunities // missed by the bottom-up traversal: let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); assign_initial_requirements(&mut sort_pushdown); let adjusted = sort_pushdown.transform_down(pushdown_sorts)?.data; + println!("AFTER sort pushdown"); + print_plan(&adjusted.plan); adjusted .plan .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?))) @@ -333,6 +348,8 @@ fn ensure_sorting( }; let plan = &requirements.plan; + // println!("ensure sorting"); + // print_plan(plan); let mut updated_children = vec![]; for (idx, (required_ordering, mut child)) in plan .required_input_ordering() @@ -341,7 +358,6 @@ fn ensure_sorting( .enumerate() { let physical_ordering = child.plan.output_ordering(); - if let Some(required) = required_ordering { let eq_properties = child.plan.equivalence_properties(); if !eq_properties.ordering_satisfy_requirement(&required) { diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 67b38dba90ca..89a7b690a82f 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -248,7 +248,7 @@ fn require_top_ordering_helper( if children.len() != 1 { Ok((plan, false)) } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let req_ordering = sort_exec.properties().output_ordering().unwrap_or(&[]); + let req_ordering = sort_exec.expr(); let req_dist = sort_exec.required_input_distribution()[0].clone(); let reqs = PhysicalSortRequirement::from_sort_exprs(req_ordering); Ok(( diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 83531da3ca8f..db72a733caf2 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -245,11 +245,38 @@ fn try_pushdown_requirements_to_join( sort_expr: Vec, push_side: JoinSide, ) -> Result>>>> { + let left_eq_properties = smj.left().equivalence_properties(); + let right_eq_properties = smj.right().equivalence_properties(); + let mut smj_required_orderings = smj.required_input_ordering(); + let right_requirement = smj_required_orderings.swap_remove(1); + let left_requirement = smj_required_orderings.swap_remove(0); let left_ordering = smj.left().output_ordering().unwrap_or(&[]); let right_ordering = smj.right().output_ordering().unwrap_or(&[]); let (new_left_ordering, new_right_ordering) = match push_side { - JoinSide::Left => (sort_expr.as_slice(), right_ordering), - JoinSide::Right => (left_ordering, sort_expr.as_slice()), + JoinSide::Left => { + let left_eq_properties = + left_eq_properties.clone().with_reorder(sort_expr.clone()); + if left_eq_properties + .ordering_satisfy_requirement(&left_requirement.unwrap_or(vec![])) + { + // After re-ordering requirement is still satisfied + (sort_expr.as_slice(), right_ordering) + } else { + return Ok(None); + } + } + JoinSide::Right => { + let right_eq_properties = + right_eq_properties.clone().with_reorder(sort_expr.clone()); + if right_eq_properties + .ordering_satisfy_requirement(&right_requirement.unwrap_or(vec![])) + { + // After re-ordering requirement is still satisfied + (left_ordering, sort_expr.as_slice()) + } else { + return Ok(None); + } + } }; let join_type = smj.join_type(); let probe_side = SortMergeJoinExec::probe_side(&join_type); diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 7faf2caae01c..542c6c2900f6 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -28,7 +28,7 @@ mod projection; mod properties; pub use class::{EquivalenceClass, EquivalenceGroup}; -pub use ordering::OrderingEquivalenceClass; +pub use ordering::{ConstExpr, OrderingEquivalenceClass}; pub use projection::ProjectionMapping; pub use properties::{join_equivalence_properties, EquivalenceProperties}; diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 7857d9df726e..5998868898cb 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -192,6 +192,21 @@ impl OrderingEquivalenceClass { } } +#[derive(Debug, Clone)] +pub struct ConstExpr { + pub expr: Arc, + pub across_partitions: bool, +} + +pub fn const_exprs_contains( + const_exprs: &[ConstExpr], + expr: &Arc, +) -> bool { + const_exprs + .iter() + .any(|const_expr| const_expr.expr.eq(expr)) +} + /// This function constructs a duplicate-free `LexOrdering` by filtering out /// duplicate entries that have same physical expression inside. For example, /// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 7bf389ecfdf3..bc0afc035595 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -18,7 +18,7 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; -use super::ordering::collapse_lex_ordering; +use super::ordering::{collapse_lex_ordering, const_exprs_contains, ConstExpr}; use crate::equivalence::{ collapse_lex_req, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, }; @@ -92,7 +92,7 @@ pub struct EquivalenceProperties { /// Expressions whose values are constant throughout the table. /// TODO: We do not need to track constants separately, they can be tracked /// inside `eq_groups` as `Literal` expressions. - pub constants: Vec>, + pub constants: Vec, /// Schema associated with this object. schema: SchemaRef, } @@ -134,7 +134,7 @@ impl EquivalenceProperties { } /// Returns a reference to the constant expressions - pub fn constants(&self) -> &[Arc] { + pub fn constants(&self) -> &[ConstExpr] { &self.constants } @@ -144,7 +144,7 @@ impl EquivalenceProperties { let mut output_ordering = self.oeq_class().output_ordering().unwrap_or_default(); // Prune out constant expressions output_ordering - .retain(|sort_expr| !physical_exprs_contains(constants, &sort_expr.expr)); + .retain(|sort_expr| !const_exprs_contains(constants, &sort_expr.expr)); (!output_ordering.is_empty()).then_some(output_ordering) } @@ -173,6 +173,10 @@ impl EquivalenceProperties { self.oeq_class.clear(); } + pub fn clear_per_partition_constants(&mut self) { + self.constants.retain(|elem| elem.across_partitions); + } + /// Extends this `EquivalenceProperties` by adding the orderings inside the /// ordering equivalence class `other`. pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { @@ -204,13 +208,19 @@ impl EquivalenceProperties { // Discover new constants in light of new the equality: if self.is_expr_constant(left) { // Left expression is constant, add right as constant - if !physical_exprs_contains(&self.constants, right) { - self.constants.push(right.clone()); + if !const_exprs_contains(&self.constants, right) { + self.constants.push(ConstExpr { + expr: right.clone(), + across_partitions: true, + }); } } else if self.is_expr_constant(right) { // Right expression is constant, add left as constant - if !physical_exprs_contains(&self.constants, left) { - self.constants.push(left.clone()); + if !const_exprs_contains(&self.constants, left) { + self.constants.push(ConstExpr { + expr: left.clone(), + across_partitions: true, + }); } } @@ -270,11 +280,33 @@ impl EquivalenceProperties { /// Track/register physical expressions with constant values. pub fn add_constants( mut self, - constants: impl IntoIterator>, + constants: impl IntoIterator, ) -> Self { - for expr in self.eq_group.normalize_exprs(constants) { - if !physical_exprs_contains(&self.constants, &expr) { - self.constants.push(expr); + let (const_exprs, across_partition_flags): ( + Vec>, + Vec, + ) = constants + .into_iter() + .map(|const_expr| { + let ConstExpr { + expr, + across_partitions, + } = const_expr; + (expr, across_partitions) + }) + .unzip(); + for (expr, across_partitions) in self + .eq_group + .normalize_exprs(const_exprs) + .into_iter() + .zip(across_partition_flags) + { + if !const_exprs_contains(&self.constants, &expr) { + let const_expr = ConstExpr { + expr, + across_partitions, + }; + self.constants.push(const_expr); } } self @@ -326,7 +358,14 @@ impl EquivalenceProperties { sort_reqs: LexRequirementRef, ) -> LexRequirement { let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); - let constants_normalized = self.eq_group.normalize_exprs(self.constants.clone()); + let mut constant_exprs = vec![]; + constant_exprs.extend( + self.constants + .iter() + .map(|const_expr| const_expr.expr.clone()) + .collect::>(), + ); + let constants_normalized = self.eq_group.normalize_exprs(constant_exprs); // Prune redundant sections in the requirement: collapse_lex_req( normalized_sort_reqs @@ -370,8 +409,10 @@ impl EquivalenceProperties { // From the analysis above, we know that `[a ASC]` is satisfied. Then, // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. - eq_properties = - eq_properties.add_constants(std::iter::once(normalized_req.expr)); + eq_properties = eq_properties.add_constants(std::iter::once(ConstExpr { + expr: normalized_req.expr, + across_partitions: false, + })); } true } @@ -781,24 +822,31 @@ impl EquivalenceProperties { /// # Returns /// /// Returns a `Vec>` containing the projected constants. - fn projected_constants( - &self, - mapping: &ProjectionMapping, - ) -> Vec> { + fn projected_constants(&self, mapping: &ProjectionMapping) -> Vec { // First, project existing constants. For example, assume that `a + b` // is known to be constant. If the projection were `a as a_new`, `b as b_new`, // then we would project constant `a + b` as `a_new + b_new`. let mut projected_constants = self .constants .iter() - .flat_map(|expr| self.eq_group.project_expr(mapping, expr)) + .flat_map(|const_expr| { + self.eq_group + .project_expr(mapping, &const_expr.expr) + .map(|expr| ConstExpr { + expr, + across_partitions: const_expr.across_partitions, + }) + }) .collect::>(); // Add projection expressions that are known to be constant: for (source, target) in mapping.iter() { if self.is_expr_constant(source) - && !physical_exprs_contains(&projected_constants, target) + && !const_exprs_contains(&projected_constants, target) { - projected_constants.push(target.clone()); + projected_constants.push(ConstExpr { + expr: target.clone(), + across_partitions: true, + }); } } projected_constants @@ -891,8 +939,10 @@ impl EquivalenceProperties { // Note that these expressions are not properly "constants". This is just // an implementation strategy confined to this function. for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { - eq_properties = - eq_properties.add_constants(std::iter::once(expr.clone())); + eq_properties = eq_properties.add_constants(std::iter::once(ConstExpr { + expr: expr.clone(), + across_partitions: false, + })); search_indices.shift_remove(idx); } // Add new ordered section to the state. @@ -917,7 +967,12 @@ impl EquivalenceProperties { // As an example, assume that we know columns `a` and `b` are constant. // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will // return `false`. - let normalized_constants = self.eq_group.normalize_exprs(self.constants.to_vec()); + let const_exprs = self + .constants + .iter() + .map(|const_expr| const_expr.expr.clone()) + .collect::>(); + let normalized_constants = self.eq_group.normalize_exprs(const_exprs); let normalized_expr = self.eq_group.normalize_expr(expr.clone()); is_constant_recurse(&normalized_constants, &normalized_expr) } diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index b764e81a95d1..cfc5213d1b76 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -46,6 +46,7 @@ pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use datafusion_physical_expr_common::aggregate::{ AggregateExpr, AggregatePhysicalExpressions, }; +pub use equivalence::ConstExpr; pub use equivalence::EquivalenceProperties; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index ce67cba2cd0e..029836c4831e 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -65,7 +65,8 @@ impl CoalescePartitionsExec { // Coalescing partitions loses existing orderings: let mut eq_properties = input.equivalence_properties().clone(); eq_properties.clear_orderings(); - + eq_properties.clear_per_partition_constants(); + println!("coalesce constants: {:?}", eq_properties.constants()); PlanProperties::new( eq_properties, // Equivalence Properties Partitioning::UnknownPartitioning(1), // Output Partitioning diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 6153dbacfbff..e5d366869625 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -44,7 +44,7 @@ use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ - analyze, split_conjunction, AnalysisContext, ExprBoundaries, PhysicalExpr, + analyze, split_conjunction, AnalysisContext, ConstExpr, ExprBoundaries, PhysicalExpr, }; use futures::stream::{Stream, StreamExt}; @@ -162,7 +162,7 @@ impl FilterExec { fn extend_constants( input: &Arc, predicate: &Arc, - ) -> Vec> { + ) -> Vec { let mut res_constants = Vec::new(); let input_eqs = input.equivalence_properties(); @@ -171,9 +171,15 @@ impl FilterExec { if let Some(binary) = conjunction.as_any().downcast_ref::() { if binary.op() == &Operator::Eq { if input_eqs.is_expr_constant(binary.left()) { - res_constants.push(binary.right().clone()) + res_constants.push(ConstExpr { + expr: binary.right().clone(), + across_partitions: true, + }) } else if input_eqs.is_expr_constant(binary.right()) { - res_constants.push(binary.left().clone()) + res_constants.push(ConstExpr { + expr: binary.left().clone(), + across_partitions: true, + }) } } } @@ -199,7 +205,13 @@ impl FilterExec { let constants = collect_columns(predicate) .into_iter() .filter(|column| stats.column_statistics[column.index()].is_singleton()) - .map(|column| Arc::new(column) as _); + .map(|column| { + let expr = Arc::new(column) as _; + ConstExpr { + expr, + across_partitions: true, + } + }); // this is for statistics eq_properties = eq_properties.add_constants(constants); // this is for logical constant (for example: a = '1', then a could be marked as a constant) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 65f7d5070a5d..a7fa2e29638a 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -701,6 +701,11 @@ impl RepartitionExec { if !Self::maintains_input_order_helper(input, preserve_order)[0] { eq_properties.clear_orderings(); } + println!("constants before: {:?}", eq_properties.constants()); + if input.output_partitioning().partition_count() > 1 { + eq_properties.clear_per_partition_constants(); + } + println!("constants after: {:?}", eq_properties.constants()); eq_properties } @@ -712,7 +717,6 @@ impl RepartitionExec { ) -> PlanProperties { // Equivalence Properties let eq_properties = Self::eq_properties_helper(input, preserve_order); - PlanProperties::new( eq_properties, // Equivalence Properties partitioning, // Output Partitioning diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 2a4862534590..e9de08de25ea 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -842,7 +842,7 @@ impl SortExec { .equivalence_properties() .clone() .with_reorder(sort_exprs); - + println!("sort eq properties: {:?}", eq_properties.constants()); // Get output partitioning: let output_partitioning = Self::output_partitioning_helper(input, preserve_partitioning); diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index dc7d270bae25..dcc52661dd75 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -41,7 +41,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::{ConstExpr, EquivalenceProperties}; use futures::Stream; use itertools::Itertools; @@ -118,41 +118,11 @@ impl UnionExec { schema: SchemaRef, ) -> PlanProperties { // Calculate equivalence properties: - // TODO: In some cases, we should be able to preserve some equivalence - // classes and constants. Add support for such cases. let children_eqs = inputs .iter() .map(|child| child.equivalence_properties()) .collect::>(); - let mut eq_properties = EquivalenceProperties::new(schema); - // Use the ordering equivalence class of the first child as the seed: - let mut meets = children_eqs[0] - .oeq_class() - .iter() - .map(|item| item.to_vec()) - .collect::>(); - // Iterate over all the children: - for child_eqs in &children_eqs[1..] { - // Compute meet orderings of the current meets and the new ordering - // equivalence class. - let mut idx = 0; - while idx < meets.len() { - // Find all the meets of `current_meet` with this child's orderings: - let valid_meets = child_eqs.oeq_class().iter().filter_map(|ordering| { - child_eqs.get_meet_ordering(ordering, &meets[idx]) - }); - // Use the longest of these meets as others are redundant: - if let Some(next_meet) = valid_meets.max_by_key(|m| m.len()) { - meets[idx] = next_meet; - idx += 1; - } else { - meets.swap_remove(idx); - } - } - } - // We know have all the valid orderings after union, remove redundant - // entries (implicitly) and return: - eq_properties.add_new_orderings(meets); + let eq_properties = calculate_union_eq_properties(&children_eqs, schema); // Calculate output partitioning; i.e. sum output partitions of the inputs. let num_partitions = inputs @@ -168,6 +138,68 @@ impl UnionExec { } } +fn calculate_union_eq_properties( + children_eqs: &[&EquivalenceProperties], + schema: SchemaRef, +) -> EquivalenceProperties { + // Calculate equivalence properties: + // TODO: In some cases, we should be able to preserve some equivalence + // classes and constants. Add support for such cases. + let mut eq_properties = EquivalenceProperties::new(schema); + // Use the ordering equivalence class of the first child as the seed: + let mut meets = children_eqs[0] + .oeq_class() + .iter() + .map(|item| item.to_vec()) + .collect::>(); + // Iterate over all the children: + for child_eqs in &children_eqs[1..] { + // Compute meet orderings of the current meets and the new ordering + // equivalence class. + let mut idx = 0; + while idx < meets.len() { + // Find all the meets of `current_meet` with this child's orderings: + let valid_meets = child_eqs.oeq_class().iter().filter_map(|ordering| { + child_eqs.get_meet_ordering(ordering, &meets[idx]) + }); + // Use the longest of these meets as others are redundant: + if let Some(next_meet) = valid_meets.max_by_key(|m| m.len()) { + meets[idx] = next_meet; + idx += 1; + } else { + meets.swap_remove(idx); + } + } + } + // We know have all the valid orderings after union, remove redundant + // entries (implicitly) and return: + eq_properties.add_new_orderings(meets); + + let mut meet_constants = children_eqs[0].constants().to_vec(); + // Iterate over all the children: + for child_eqs in &children_eqs[1..] { + let constants = child_eqs.constants(); + meet_constants = meet_constants + .into_iter() + .filter_map(|meet_constant| { + for const_expr in constants { + if const_expr.expr.eq(&meet_constant.expr) { + // TODO: Check whether constant expressions evaluates the same value or not for each partition + let across_partitions = false; + return Some(ConstExpr { + expr: meet_constant.expr, + across_partitions, + }); + } + } + None + }) + .collect::>(); + } + eq_properties = eq_properties.add_constants(meet_constants); + eq_properties +} + impl DisplayAs for UnionExec { fn fmt_as( &self, diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index ecfe123a43af..71cacf2de43d 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -41,7 +41,8 @@ use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, - AggregateExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, + AggregateExpr, ConstExpr, EquivalenceProperties, LexOrdering, + PhysicalSortRequirement, }; use itertools::Itertools; @@ -572,7 +573,14 @@ pub fn get_window_mode( options: None, })); // Treat partition by exprs as constant. During analysis of requirements are satisfied. - let partition_by_eqs = input_eqs.add_constants(partitionby_exprs.iter().cloned()); + let const_exprs = partitionby_exprs + .iter() + .map(|expr| ConstExpr { + expr: expr.clone(), + across_partitions: false, + }) + .collect::>(); + let partition_by_eqs = input_eqs.add_constants(const_exprs); let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); let reverse_order_by_reqs = PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(orderby_keys)); diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 378cab206240..b982c6899317 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -251,7 +251,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id], aggr=[ARRAY_AGG(DISTINCT a.foo), sum(DISTINCT Int64(1))] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------RepartitionExec: partitioning=Hash([id@0], 4), input_partitions=5 -05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[ARRAY_AGG(DISTINCT a.foo), sum(DISTINCT Int64(1))] +05)--------AggregateExec: mode=Partial, gby=[id@0 as id], aggr=[ARRAY_AGG(DISTINCT a.foo), sum(DISTINCT Int64(1))], ordering_mode=Sorted 06)----------UnionExec 07)------------ProjectionExec: expr=[1 as id, 2 as foo] 08)--------------PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index ea7103255697..0aafa1d64a88 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3188,19 +3188,20 @@ logical_plan 08)------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] -02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -03)----SortExec: expr=[rn1@5 ASC NULLS LAST], preserve_partitioning=[true] -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------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] -08)--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -09)----------------CsvExec: 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], has_header=true -10)----SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -11)------CoalesceBatchesExec: target_batch_size=2 -12)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -13)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)------------CsvExec: 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], has_header=true +02)--SortExec: expr=[rn1@5 ASC NULLS LAST], preserve_partitioning=[true] +03)----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------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] +09)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +10)------------------CsvExec: 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], has_header=true +11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] +12)--------CoalesceBatchesExec: target_batch_size=2 +13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +15)--------------CsvExec: 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], has_header=true # sort merge join should propagate ordering equivalence of the right side # for right join. Hence final requirement rn1 ASC is already satisfied at @@ -3224,19 +3225,20 @@ logical_plan 08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] -02)--SortMergeJoin: join_type=Right, on=[(a@1, a@1)] -03)----SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------CsvExec: 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], has_header=true -08)----SortExec: expr=[rn1@5 ASC NULLS LAST], preserve_partitioning=[true] -09)------CoalesceBatchesExec: target_batch_size=2 -10)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -11)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)------------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] -13)--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -14)----------------CsvExec: 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], has_header=true +02)--SortExec: expr=[rn1@10 ASC NULLS LAST], preserve_partitioning=[true] +03)----SortMergeJoin: join_type=Right, on=[(a@1, a@1)] +04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------CsvExec: 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], has_header=true +09)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] +10)--------CoalesceBatchesExec: target_batch_size=2 +11)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)--------------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] +14)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +15)------------------CsvExec: 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], has_header=true # SortMergeJoin should add ordering equivalences of # right table as lexicographical append to the global ordering @@ -3281,6 +3283,8 @@ physical_plan 16)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 17)------------------CsvExec: 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], has_header=true + + statement ok set datafusion.optimizer.prefer_hash_join = true; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index ded20f513364..c5b8d0ebbc71 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -276,7 +276,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a)] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)] +07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)], ordering_mode=Sorted 08)--------------UnionExec 09)----------------ProjectionExec: expr=[1 as a, aa as b] 10)------------------PlaceholderRowExec From c08a92b51cf2bb1d3358e571ad1c7460b58bb888 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jun 2024 16:21:46 +0300 Subject: [PATCH 12/28] Slt tests pass --- .../src/physical_optimizer/sanity_checker.rs | 16 +++++++++++----- .../physical-expr/src/equivalence/properties.rs | 13 +++++++++++++ 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 52a8581d8ced..efd9b365af4a 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -33,7 +33,7 @@ use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; -use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::{displayable, ExecutionPlanProperties}; /// The SanityCheckPlan rule rejects the following query plans: /// i) Plans that use pipeline-breaking operators on infinite input(s), @@ -112,6 +112,12 @@ fn is_prunable(join: &SymmetricHashJoinExec) -> bool { }) } +fn print_plan(plan: &Arc) { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + /// Ensures that the plan is pipeline friendly and the order and /// distribution requirements from its children are satisfied. pub fn check_plan_sanity( @@ -131,8 +137,8 @@ pub fn check_plan_sanity( Some(child_sort_req) => { if !child_eq_props.ordering_satisfy_requirement(child_sort_req) { return plan_err!( - "Child: {:?} does not satisfy parent order requirements", - child + "Child does not satisfy parent order requirements: {:?}", + child_sort_req ); } } @@ -143,8 +149,8 @@ pub fn check_plan_sanity( .satisfy(child_dist_req, child_eq_props) { return plan_err!( - "Child: {:?} does not satisfy parent distribution requirements", - child + "Child does not satisfy parent distribution requirements: {:?}", + child_dist_req ); } } diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index bc0afc035595..ae7e3a97222f 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -394,6 +394,8 @@ impl EquivalenceProperties { let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); for normalized_req in normalized_reqs { // Check whether given ordering is satisfied + // println!("normalized_req: {:?}", normalized_req); + // println!("eq_properties: {:?}", eq_properties); if !eq_properties.ordering_satisfy_single(&normalized_req) { return false; } @@ -1362,6 +1364,8 @@ pub fn join_equivalence_properties( on, )); + let left_constants = left.constants().to_vec(); + let right_constants = right.constants().to_vec(); let left_oeq_class = left.oeq_class; let mut right_oeq_class = right.oeq_class; match maintains_input_order { @@ -1416,6 +1420,15 @@ pub fn join_equivalence_properties( [true, true] => unreachable!("Cannot maintain ordering of both sides"), _ => unreachable!("Join operators can not have more than two children"), } + match join_type { + JoinType::LeftAnti | JoinType::LeftSemi => { + result = result.add_constants(left_constants); + } + JoinType::RightAnti | JoinType::RightSemi => { + result = result.add_constants(right_constants); + } + _ => {} + } result } From 6d79881abe9c92c3f8b038f1c60c98e9a5c84a04 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jun 2024 16:32:48 +0300 Subject: [PATCH 13/28] Resolve linter errors --- .../src/physical_optimizer/enforce_sorting.rs | 20 +------------------ .../src/physical_optimizer/sanity_checker.rs | 8 +------- .../src/physical_optimizer/sort_pushdown.rs | 4 ++-- .../sort_preserving_repartition_fuzz.rs | 9 ++++++--- .../physical-expr/src/equivalence/mod.rs | 7 +++++-- .../physical-expr/src/equivalence/ordering.rs | 10 ++++++++-- .../src/equivalence/properties.rs | 13 ++++++++++-- .../physical-plan/src/repartition/mod.rs | 2 -- 8 files changed, 34 insertions(+), 39 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 7d6f34fcd17e..0c7a46b67dd8 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -64,7 +64,7 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; -use datafusion_physical_plan::{displayable, ExecutionPlanProperties}; +use datafusion_physical_plan::ExecutionPlanProperties; use itertools::izip; @@ -147,12 +147,6 @@ fn update_coalesce_ctx_children( }; } -fn print_plan(plan: &Arc) { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - // println!("{:#?}", actual); -} - /// The boolean flag `repartition_sorts` defined in the config indicates /// whether we elect to transform [`CoalescePartitionsExec`] + [`SortExec`] cascades /// into [`SortExec`] + [`SortPreservingMergeExec`] cascades, which enables us to @@ -163,14 +157,10 @@ impl PhysicalOptimizerRule for EnforceSorting { plan: Arc, config: &ConfigOptions, ) -> Result> { - println!("start enforce req"); - print_plan(&plan); let plan_requirements = PlanWithCorrespondingSort::new_default(plan); // Execute a bottom-up traversal to enforce sorting requirements, // remove unnecessary sorts, and optimize sort-sensitive operators: let adjusted = plan_requirements.transform_up(ensure_sorting)?.data; - println!("AFTER enforce req"); - print_plan(&adjusted.plan); let new_plan = if config.optimizer.repartition_sorts { let plan_with_coalesce_partitions = PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan); @@ -181,8 +171,6 @@ impl PhysicalOptimizerRule for EnforceSorting { } else { adjusted.plan }; - println!("AFTER parallelize"); - print_plan(&new_plan); let plan_with_pipeline_fixer = OrderPreservationContext::new_default(new_plan); let updated_plan = plan_with_pipeline_fixer @@ -195,16 +183,12 @@ impl PhysicalOptimizerRule for EnforceSorting { ) }) .data()?; - println!("AFTER replace order preserving variant"); - print_plan(&updated_plan.plan); // Execute a top-down traversal to exploit sort push-down opportunities // missed by the bottom-up traversal: let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); assign_initial_requirements(&mut sort_pushdown); let adjusted = sort_pushdown.transform_down(pushdown_sorts)?.data; - println!("AFTER sort pushdown"); - print_plan(&adjusted.plan); adjusted .plan .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?))) @@ -348,8 +332,6 @@ fn ensure_sorting( }; let plan = &requirements.plan; - // println!("ensure sorting"); - // print_plan(plan); let mut updated_children = vec![]; for (idx, (required_ordering, mut child)) in plan .required_input_ordering() diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index efd9b365af4a..57647834a082 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -33,7 +33,7 @@ use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; -use datafusion_physical_plan::{displayable, ExecutionPlanProperties}; +use datafusion_physical_plan::ExecutionPlanProperties; /// The SanityCheckPlan rule rejects the following query plans: /// i) Plans that use pipeline-breaking operators on infinite input(s), @@ -112,12 +112,6 @@ fn is_prunable(join: &SymmetricHashJoinExec) -> bool { }) } -fn print_plan(plan: &Arc) { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - /// Ensures that the plan is pipeline friendly and the order and /// distribution requirements from its children are satisfied. pub fn check_plan_sanity( diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index db72a733caf2..36ac4b22d594 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -257,7 +257,7 @@ fn try_pushdown_requirements_to_join( let left_eq_properties = left_eq_properties.clone().with_reorder(sort_expr.clone()); if left_eq_properties - .ordering_satisfy_requirement(&left_requirement.unwrap_or(vec![])) + .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) { // After re-ordering requirement is still satisfied (sort_expr.as_slice(), right_ordering) @@ -269,7 +269,7 @@ fn try_pushdown_requirements_to_join( let right_eq_properties = right_eq_properties.clone().with_reorder(sort_expr.clone()); if right_eq_properties - .ordering_satisfy_requirement(&right_requirement.unwrap_or(vec![])) + .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) { // After re-ordering requirement is still satisfied (left_ordering, sort_expr.as_slice()) diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 21ef8a7c2110..c25a6b5125ab 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -41,7 +41,7 @@ mod sp_repartition_fuzz_tests { use datafusion_physical_expr::{ equivalence::{EquivalenceClass, EquivalenceProperties}, expressions::{col, Column}, - PhysicalExpr, PhysicalSortExpr, + ConstExpr, PhysicalExpr, PhysicalSortExpr, }; use test_utils::add_empty_batches; @@ -80,7 +80,10 @@ mod sp_repartition_fuzz_tests { // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f)?; // Column e has constant value. - eq_properties = eq_properties.add_constants([col_e.clone()]); + eq_properties = eq_properties.add_constants([ConstExpr { + expr: col_e.clone(), + across_partitions: false, + }]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); @@ -149,7 +152,7 @@ mod sp_repartition_fuzz_tests { // Fill constant columns for constant in eq_properties.constants() { - let col = constant.as_any().downcast_ref::().unwrap(); + let col = constant.expr.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); let arr = Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 542c6c2900f6..3784f2c006dc 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -205,7 +205,10 @@ mod tests { // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f)?; // Column e has constant value. - eq_properties = eq_properties.add_constants([col_e.clone()]); + eq_properties = eq_properties.add_constants([ConstExpr { + expr: col_e.clone(), + across_partitions: false, + }]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); @@ -482,7 +485,7 @@ mod tests { // Fill constant columns for constant in &eq_properties.constants { - let col = constant.as_any().downcast_ref::().unwrap(); + let col = constant.expr.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); let arr = Arc::new(Float64Array::from_iter_values(vec![0 as f64; n_elem])) as ArrayRef; diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 5998868898cb..b289de58035f 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -249,7 +249,7 @@ mod tests { }; use crate::expressions::{col, BinaryExpr, Column}; use crate::utils::tests::TestScalarUDF; - use crate::{PhysicalExpr, PhysicalSortExpr}; + use crate::{ConstExpr, PhysicalExpr, PhysicalSortExpr}; use arrow::datatypes::{DataType, Field, Schema}; use arrow_schema::SortOptions; @@ -569,7 +569,13 @@ mod tests { let eq_group = EquivalenceGroup::new(eq_group); eq_properties.add_equivalence_group(eq_group); - let constants = constants.into_iter().cloned(); + let constants = constants + .into_iter() + .map(|expr| ConstExpr { + expr: expr.clone(), + across_partitions: true, + }) + .collect::>(); eq_properties = eq_properties.add_constants(constants); let reqs = convert_to_sort_exprs(&reqs); diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index ae7e3a97222f..5d4ba9b8bc5a 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -2156,7 +2156,10 @@ mod tests { let col_h = &col("h", &test_schema)?; // Add column h as constant - eq_properties = eq_properties.add_constants(vec![col_h.clone()]); + eq_properties = eq_properties.add_constants(vec![ConstExpr { + expr: col_h.clone(), + across_partitions: false, + }]); let test_cases = vec![ // TEST CASE 1 @@ -2454,7 +2457,13 @@ mod tests { ]; for case in cases { - let mut properties = base_properties.clone().add_constants(case.constants); + let mut properties = + base_properties + .clone() + .add_constants(case.constants.into_iter().map(|expr| ConstExpr { + expr, + across_partitions: false, + })); for [left, right] in &case.equal_conditions { properties.add_equal_conditions(left, right)? } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index a7fa2e29638a..3e020650ecad 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -701,11 +701,9 @@ impl RepartitionExec { if !Self::maintains_input_order_helper(input, preserve_order)[0] { eq_properties.clear_orderings(); } - println!("constants before: {:?}", eq_properties.constants()); if input.output_partitioning().partition_count() > 1 { eq_properties.clear_per_partition_constants(); } - println!("constants after: {:?}", eq_properties.constants()); eq_properties } From cd49e2e8ec378b4b7d8b9b022f1d36647a3d5355 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jun 2024 17:36:34 +0300 Subject: [PATCH 14/28] Minor changes --- .../src/physical_optimizer/enforce_sorting.rs | 2 + .../sort_preserving_repartition_fuzz.rs | 7 +- .../physical-expr/src/equivalence/class.rs | 55 +++++++++++++ .../physical-expr/src/equivalence/mod.rs | 11 +-- .../physical-expr/src/equivalence/ordering.rs | 20 +---- .../src/equivalence/properties.rs | 81 +++++++------------ .../physical-plan/src/coalesce_partitions.rs | 1 - datafusion/physical-plan/src/filter.rs | 22 +++-- .../physical-plan/src/repartition/mod.rs | 1 + datafusion/physical-plan/src/sorts/sort.rs | 2 +- datafusion/physical-plan/src/union.rs | 10 +-- datafusion/physical-plan/src/windows/mod.rs | 5 +- datafusion/sqllogictest/test_files/joins.slt | 2 - 13 files changed, 113 insertions(+), 106 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 0c7a46b67dd8..24306647c686 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -183,6 +183,7 @@ impl PhysicalOptimizerRule for EnforceSorting { ) }) .data()?; + // Execute a top-down traversal to exploit sort push-down opportunities // missed by the bottom-up traversal: let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); @@ -340,6 +341,7 @@ fn ensure_sorting( .enumerate() { let physical_ordering = child.plan.output_ordering(); + if let Some(required) = required_ordering { let eq_properties = child.plan.equivalence_properties(); if !eq_properties.ordering_satisfy_requirement(&required) { diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index c25a6b5125ab..f00d17a06ffc 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -80,10 +80,7 @@ mod sp_repartition_fuzz_tests { // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f)?; // Column e has constant value. - eq_properties = eq_properties.add_constants([ConstExpr { - expr: col_e.clone(), - across_partitions: false, - }]); + eq_properties = eq_properties.add_constants([ConstExpr::new(col_e.clone())]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); @@ -152,7 +149,7 @@ mod sp_repartition_fuzz_tests { // Fill constant columns for constant in eq_properties.constants() { - let col = constant.expr.as_any().downcast_ref::().unwrap(); + let col = constant.expr().as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); let arr = Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index b4d12e963611..265b20582575 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -28,6 +28,61 @@ use crate::{ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::JoinType; +#[derive(Debug, Clone)] +/// A structure representing a constant expression in a physical execution plan. +/// +/// The `ConstExpr` struct encapsulates an expression that is constant during the execution +/// of a query. +/// +/// # Fields +/// +/// - `expr`: Constant expression for a node in the physical plan. +/// +/// - `across_partitions`: A boolean flag indicating whether the constant expression is +/// valid across partitions. If set to `true`, the constant expression has same value for all partitions. +/// If set to `false`, the constant expression may have different values for different partitions. +pub struct ConstExpr { + expr: Arc, + across_partitions: bool, +} + +impl ConstExpr { + pub fn new(expr: Arc) -> Self { + Self { + expr, + // By default, assume constant expressions are not same accross partitions. + across_partitions: false, + } + } + + pub fn with_across_partitions(mut self, across_partitions: bool) -> Self { + self.across_partitions = across_partitions; + self + } + + pub fn across_partitions(&self) -> bool { + self.across_partitions + } + + pub fn expr(&self) -> &Arc { + &self.expr + } + + pub fn owned_expr(self) -> Arc { + self.expr + } +} + +/// Checks whether `expr` is among in the `const_exprs`. +pub fn const_exprs_contains( + const_exprs: &[ConstExpr], + expr: &Arc, +) -> bool { + const_exprs + .iter() + .any(|const_expr| const_expr.expr.eq(expr)) +} + /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by /// equality predicates (e.g. `a = b`), typically equi-join conditions and diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 3784f2c006dc..5eb8a19e3d67 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -27,8 +27,8 @@ mod ordering; mod projection; mod properties; -pub use class::{EquivalenceClass, EquivalenceGroup}; -pub use ordering::{ConstExpr, OrderingEquivalenceClass}; +pub use class::{ConstExpr, EquivalenceClass, EquivalenceGroup}; +pub use ordering::OrderingEquivalenceClass; pub use projection::ProjectionMapping; pub use properties::{join_equivalence_properties, EquivalenceProperties}; @@ -205,10 +205,7 @@ mod tests { // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f)?; // Column e has constant value. - eq_properties = eq_properties.add_constants([ConstExpr { - expr: col_e.clone(), - across_partitions: false, - }]); + eq_properties = eq_properties.add_constants([ConstExpr::new(col_e.clone())]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); @@ -485,7 +482,7 @@ mod tests { // Fill constant columns for constant in &eq_properties.constants { - let col = constant.expr.as_any().downcast_ref::().unwrap(); + let col = constant.expr().as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); let arr = Arc::new(Float64Array::from_iter_values(vec![0 as f64; n_elem])) as ArrayRef; diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index b289de58035f..7899e6334c8d 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -192,21 +192,6 @@ impl OrderingEquivalenceClass { } } -#[derive(Debug, Clone)] -pub struct ConstExpr { - pub expr: Arc, - pub across_partitions: bool, -} - -pub fn const_exprs_contains( - const_exprs: &[ConstExpr], - expr: &Arc, -) -> bool { - const_exprs - .iter() - .any(|const_expr| const_expr.expr.eq(expr)) -} - /// This function constructs a duplicate-free `LexOrdering` by filtering out /// duplicate entries that have same physical expression inside. For example, /// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. @@ -571,10 +556,7 @@ mod tests { let constants = constants .into_iter() - .map(|expr| ConstExpr { - expr: expr.clone(), - across_partitions: true, - }) + .map(|expr| ConstExpr::new(expr.clone()).with_across_partitions(true)) .collect::>(); eq_properties = eq_properties.add_constants(constants); diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 5d4ba9b8bc5a..2e07fa6315c1 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -18,13 +18,13 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; -use super::ordering::{collapse_lex_ordering, const_exprs_contains, ConstExpr}; +use super::ordering::collapse_lex_ordering; use crate::equivalence::{ collapse_lex_req, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, }; use crate::expressions::Literal; use crate::{ - physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, + physical_exprs_contains, ConstExpr, LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement, }; @@ -38,6 +38,7 @@ use datafusion_physical_expr_common::expressions::column::Column; use datafusion_physical_expr_common::expressions::CastExpr; use datafusion_physical_expr_common::utils::ExprPropertiesNode; +use crate::equivalence::class::const_exprs_contains; use indexmap::{IndexMap, IndexSet}; use itertools::Itertools; @@ -173,8 +174,10 @@ impl EquivalenceProperties { self.oeq_class.clear(); } + /// Removes constant expressions that may change across partitions. + /// (Should be used when different partitions are fused) pub fn clear_per_partition_constants(&mut self) { - self.constants.retain(|elem| elem.across_partitions); + self.constants.retain(|elem| elem.across_partitions()); } /// Extends this `EquivalenceProperties` by adding the orderings inside the @@ -209,18 +212,14 @@ impl EquivalenceProperties { if self.is_expr_constant(left) { // Left expression is constant, add right as constant if !const_exprs_contains(&self.constants, right) { - self.constants.push(ConstExpr { - expr: right.clone(), - across_partitions: true, - }); + self.constants + .push(ConstExpr::new(right.clone()).with_across_partitions(true)); } } else if self.is_expr_constant(right) { // Right expression is constant, add left as constant if !const_exprs_contains(&self.constants, left) { - self.constants.push(ConstExpr { - expr: left.clone(), - across_partitions: true, - }); + self.constants + .push(ConstExpr::new(left.clone()).with_across_partitions(true)); } } @@ -288,10 +287,8 @@ impl EquivalenceProperties { ) = constants .into_iter() .map(|const_expr| { - let ConstExpr { - expr, - across_partitions, - } = const_expr; + let across_partitions = const_expr.across_partitions(); + let expr = const_expr.owned_expr(); (expr, across_partitions) }) .unzip(); @@ -302,10 +299,8 @@ impl EquivalenceProperties { .zip(across_partition_flags) { if !const_exprs_contains(&self.constants, &expr) { - let const_expr = ConstExpr { - expr, - across_partitions, - }; + let const_expr = + ConstExpr::new(expr).with_across_partitions(across_partitions); self.constants.push(const_expr); } } @@ -362,7 +357,7 @@ impl EquivalenceProperties { constant_exprs.extend( self.constants .iter() - .map(|const_expr| const_expr.expr.clone()) + .map(|const_expr| const_expr.expr().clone()) .collect::>(), ); let constants_normalized = self.eq_group.normalize_exprs(constant_exprs); @@ -394,8 +389,6 @@ impl EquivalenceProperties { let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); for normalized_req in normalized_reqs { // Check whether given ordering is satisfied - // println!("normalized_req: {:?}", normalized_req); - // println!("eq_properties: {:?}", eq_properties); if !eq_properties.ordering_satisfy_single(&normalized_req) { return false; } @@ -411,10 +404,8 @@ impl EquivalenceProperties { // From the analysis above, we know that `[a ASC]` is satisfied. Then, // we add column `a` as constant to the algorithm state. This enables us // to deduce that `(b + c) ASC` is satisfied, given `a` is constant. - eq_properties = eq_properties.add_constants(std::iter::once(ConstExpr { - expr: normalized_req.expr, - across_partitions: false, - })); + eq_properties = eq_properties + .add_constants(std::iter::once(ConstExpr::new(normalized_req.expr))); } true } @@ -833,10 +824,10 @@ impl EquivalenceProperties { .iter() .flat_map(|const_expr| { self.eq_group - .project_expr(mapping, &const_expr.expr) - .map(|expr| ConstExpr { - expr, - across_partitions: const_expr.across_partitions, + .project_expr(mapping, const_expr.expr()) + .map(|expr| { + ConstExpr::new(expr) + .with_across_partitions(const_expr.across_partitions()) }) }) .collect::>(); @@ -845,10 +836,9 @@ impl EquivalenceProperties { if self.is_expr_constant(source) && !const_exprs_contains(&projected_constants, target) { - projected_constants.push(ConstExpr { - expr: target.clone(), - across_partitions: true, - }); + // Expression evaluates to single value + projected_constants + .push(ConstExpr::new(target.clone()).with_across_partitions(true)); } } projected_constants @@ -941,10 +931,8 @@ impl EquivalenceProperties { // Note that these expressions are not properly "constants". This is just // an implementation strategy confined to this function. for (PhysicalSortExpr { expr, .. }, idx) in &ordered_exprs { - eq_properties = eq_properties.add_constants(std::iter::once(ConstExpr { - expr: expr.clone(), - across_partitions: false, - })); + eq_properties = eq_properties + .add_constants(std::iter::once(ConstExpr::new(expr.clone()))); search_indices.shift_remove(idx); } // Add new ordered section to the state. @@ -972,7 +960,7 @@ impl EquivalenceProperties { let const_exprs = self .constants .iter() - .map(|const_expr| const_expr.expr.clone()) + .map(|const_expr| const_expr.expr().clone()) .collect::>(); let normalized_constants = self.eq_group.normalize_exprs(const_exprs); let normalized_expr = self.eq_group.normalize_expr(expr.clone()); @@ -2156,10 +2144,7 @@ mod tests { let col_h = &col("h", &test_schema)?; // Add column h as constant - eq_properties = eq_properties.add_constants(vec![ConstExpr { - expr: col_h.clone(), - across_partitions: false, - }]); + eq_properties = eq_properties.add_constants(vec![ConstExpr::new(col_h.clone())]); let test_cases = vec![ // TEST CASE 1 @@ -2457,13 +2442,9 @@ mod tests { ]; for case in cases { - let mut properties = - base_properties - .clone() - .add_constants(case.constants.into_iter().map(|expr| ConstExpr { - expr, - across_partitions: false, - })); + let mut properties = base_properties + .clone() + .add_constants(case.constants.into_iter().map(ConstExpr::new)); for [left, right] in &case.equal_conditions { properties.add_equal_conditions(left, right)? } diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 029836c4831e..93f449f2d39b 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -66,7 +66,6 @@ impl CoalescePartitionsExec { let mut eq_properties = input.equivalence_properties().clone(); eq_properties.clear_orderings(); eq_properties.clear_per_partition_constants(); - println!("coalesce constants: {:?}", eq_properties.constants()); PlanProperties::new( eq_properties, // Equivalence Properties Partitioning::UnknownPartitioning(1), // Output Partitioning diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index e5d366869625..c141958c1171 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -170,16 +170,17 @@ impl FilterExec { for conjunction in conjunctions { if let Some(binary) = conjunction.as_any().downcast_ref::() { if binary.op() == &Operator::Eq { + // Filter evaluates to single value for all partitions if input_eqs.is_expr_constant(binary.left()) { - res_constants.push(ConstExpr { - expr: binary.right().clone(), - across_partitions: true, - }) + res_constants.push( + ConstExpr::new(binary.right().clone()) + .with_across_partitions(true), + ) } else if input_eqs.is_expr_constant(binary.right()) { - res_constants.push(ConstExpr { - expr: binary.left().clone(), - across_partitions: true, - }) + res_constants.push( + ConstExpr::new(binary.left().clone()) + .with_across_partitions(true), + ) } } } @@ -207,10 +208,7 @@ impl FilterExec { .filter(|column| stats.column_statistics[column.index()].is_singleton()) .map(|column| { let expr = Arc::new(column) as _; - ConstExpr { - expr, - across_partitions: true, - } + ConstExpr::new(expr).with_across_partitions(true) }); // this is for statistics eq_properties = eq_properties.add_constants(constants); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 3e020650ecad..c10dff23d034 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -715,6 +715,7 @@ impl RepartitionExec { ) -> PlanProperties { // Equivalence Properties let eq_properties = Self::eq_properties_helper(input, preserve_order); + PlanProperties::new( eq_properties, // Equivalence Properties partitioning, // Output Partitioning diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index e9de08de25ea..2a4862534590 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -842,7 +842,7 @@ impl SortExec { .equivalence_properties() .clone() .with_reorder(sort_exprs); - println!("sort eq properties: {:?}", eq_properties.constants()); + // Get output partitioning: let output_partitioning = Self::output_partitioning_helper(input, preserve_partitioning); diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index dcc52661dd75..ffd65adb8ae4 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -183,13 +183,13 @@ fn calculate_union_eq_properties( .into_iter() .filter_map(|meet_constant| { for const_expr in constants { - if const_expr.expr.eq(&meet_constant.expr) { + if const_expr.expr().eq(meet_constant.expr()) { // TODO: Check whether constant expressions evaluates the same value or not for each partition let across_partitions = false; - return Some(ConstExpr { - expr: meet_constant.expr, - across_partitions, - }); + return Some( + ConstExpr::new(meet_constant.owned_expr()) + .with_across_partitions(across_partitions), + ); } } None diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 71cacf2de43d..4710303cfa41 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -575,10 +575,7 @@ pub fn get_window_mode( // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs .iter() - .map(|expr| ConstExpr { - expr: expr.clone(), - across_partitions: false, - }) + .map(|expr| ConstExpr::new(expr.clone())) .collect::>(); let partition_by_eqs = input_eqs.add_constants(const_exprs); let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 0aafa1d64a88..da741de39a11 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3283,8 +3283,6 @@ physical_plan 16)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 17)------------------CsvExec: 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], has_header=true - - statement ok set datafusion.optimizer.prefer_hash_join = true; From 338f4517143afa2d325a96b4c745aa51b9edb95e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jun 2024 17:40:29 +0300 Subject: [PATCH 15/28] Minor changes --- .../physical-plan/src/sorts/sort_preserving_merge.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 8a349bd22abf..b19e618ae7d5 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -112,10 +112,12 @@ impl SortPreservingMergeExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties(input: &Arc) -> PlanProperties { + let mut eq_properties = input.equivalence_properties().clone(); + eq_properties.clear_per_partition_constants(); PlanProperties::new( - input.equivalence_properties().clone(), // Equivalence Properties - Partitioning::UnknownPartitioning(1), // Output Partitioning - input.execution_mode(), // Execution Mode + eq_properties, // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + input.execution_mode(), // Execution Mode ) } } From 1959488efea936a0862d0eeaf3ac0e496ece1656 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jun 2024 17:44:25 +0300 Subject: [PATCH 16/28] Minor changes --- .../physical-plan/src/sorts/sort_preserving_merge.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index b19e618ae7d5..8a349bd22abf 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -112,12 +112,10 @@ impl SortPreservingMergeExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties(input: &Arc) -> PlanProperties { - let mut eq_properties = input.equivalence_properties().clone(); - eq_properties.clear_per_partition_constants(); PlanProperties::new( - eq_properties, // Equivalence Properties - Partitioning::UnknownPartitioning(1), // Output Partitioning - input.execution_mode(), // Execution Mode + input.equivalence_properties().clone(), // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + input.execution_mode(), // Execution Mode ) } } From 1c5c0c0bbab30a40c80bce044779db81720c2a22 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jun 2024 17:50:43 +0300 Subject: [PATCH 17/28] Minor changes --- .../physical-plan/src/repartition/mod.rs | 2 + datafusion/sqllogictest/test_files/joins.slt | 56 +++++++++---------- 2 files changed, 30 insertions(+), 28 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c10dff23d034..a8029132682c 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -701,6 +701,8 @@ impl RepartitionExec { if !Self::maintains_input_order_helper(input, preserve_order)[0] { eq_properties.clear_orderings(); } + // When input partition is larger than 1, partitions will be fused at the output. + // Remove per partition constants. if input.output_partitioning().partition_count() > 1 { eq_properties.clear_per_partition_constants(); } diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index da741de39a11..96351de3919c 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3166,6 +3166,9 @@ WITH ORDER (a ASC NULLS FIRST, b ASC, c ASC) LOCATION '../core/tests/data/window_2.csv' OPTIONS ('format.has_header' 'true'); +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + # sort merge join should propagate ordering equivalence of the left side # for inner join. Hence final requirement rn1 ASC is already satisfied at # the end of SortMergeJoinExec. @@ -3188,20 +3191,17 @@ logical_plan 08)------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] -02)--SortExec: expr=[rn1@5 ASC NULLS LAST], preserve_partitioning=[true] -03)----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------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] -09)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -10)------------------CsvExec: 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], has_header=true -11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -12)--------CoalesceBatchesExec: target_batch_size=2 -13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)--------------CsvExec: 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], has_header=true +02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@5 ASC NULLS LAST +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------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] +07)------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +08)--------------CsvExec: 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], has_header=true +09)----CoalesceBatchesExec: target_batch_size=2 +10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST +11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------CsvExec: 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], has_header=true # sort merge join should propagate ordering equivalence of the right side # for right join. Hence final requirement rn1 ASC is already satisfied at @@ -3225,20 +3225,20 @@ logical_plan 08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan 01)SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] -02)--SortExec: expr=[rn1@10 ASC NULLS LAST], preserve_partitioning=[true] -03)----SortMergeJoin: join_type=Right, on=[(a@1, a@1)] -04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------CsvExec: 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], has_header=true -09)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] -10)--------CoalesceBatchesExec: target_batch_size=2 -11)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)--------------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] -14)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -15)------------------CsvExec: 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], has_header=true +02)--SortMergeJoin: join_type=Right, on=[(a@1, a@1)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CsvExec: 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], has_header=true +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@5 ASC NULLS LAST +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------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] +11)------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(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: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +12)--------------CsvExec: 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], has_header=true + +statement ok +set datafusion.optimizer.prefer_existing_sort = false; # SortMergeJoin should add ordering equivalences of # right table as lexicographical append to the global ordering From 24e32b6880f8e38873f0cd1dd8f5a75c243a1132 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 25 Jun 2024 21:25:51 +0300 Subject: [PATCH 18/28] Sort PreservingMerge clear per partition --- .../src/sorts/sort_preserving_merge.rs | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 8a349bd22abf..e364aca3791c 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -80,7 +80,7 @@ pub struct SortPreservingMergeExec { impl SortPreservingMergeExec { /// Create a new sort execution plan pub fn new(expr: Vec, input: Arc) -> Self { - let cache = Self::compute_properties(&input); + let cache = Self::compute_properties(&input, expr.clone()); Self { input, expr, @@ -111,11 +111,17 @@ impl SortPreservingMergeExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties(input: &Arc) -> PlanProperties { + fn compute_properties( + input: &Arc, + ordering: Vec, + ) -> PlanProperties { + let mut eq_properties = input.equivalence_properties().clone(); + eq_properties.clear_per_partition_constants(); + eq_properties.add_new_orderings(vec![ordering]); PlanProperties::new( - input.equivalence_properties().clone(), // Equivalence Properties - Partitioning::UnknownPartitioning(1), // Output Partitioning - input.execution_mode(), // Execution Mode + eq_properties, // Equivalence Properties + Partitioning::UnknownPartitioning(1), // Output Partitioning + input.execution_mode(), // Execution Mode ) } } From 9b77608902b38ba0049a4a6a7370b5090139e039 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 26 Jun 2024 08:57:25 +0300 Subject: [PATCH 19/28] Minor changes --- .../src/physical_optimizer/sanity_checker.rs | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 52a8581d8ced..e40e6bb21383 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -21,7 +21,6 @@ //! infinite input(s). In addition, it will check if all order and //! distribution requirements of a plan are satisfied by its children. -use itertools::izip; use std::sync::Arc; use crate::error::Result; @@ -33,11 +32,13 @@ use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; -use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties}; + +use itertools::izip; /// The SanityCheckPlan rule rejects the following query plans: /// i) Plans that use pipeline-breaking operators on infinite input(s), -/// these queries cannot be ran. +/// these queries cannot be executed. /// ii) Plans in which their order and distribution requirements are not /// satisfied by its children, these queries will most likely yield /// incorrect results. @@ -130,9 +131,11 @@ pub fn check_plan_sanity( None => (), Some(child_sort_req) => { if !child_eq_props.ordering_satisfy_requirement(child_sort_req) { + let child_plan_str = get_plan_string(child); return plan_err!( - "Child: {:?} does not satisfy parent order requirements", - child + "Child: {:?} does not satisfy parent order requirements: {:?}", + child_plan_str, + child_sort_req ); } } @@ -142,9 +145,11 @@ pub fn check_plan_sanity( .output_partitioning() .satisfy(child_dist_req, child_eq_props) { + let child_plan_str = get_plan_string(child); return plan_err!( - "Child: {:?} does not satisfy parent distribution requirements", - child + "Child: {:?} does not satisfy parent distribution requirements: {:?}", + child_plan_str, + child_dist_req ); } } @@ -462,6 +467,7 @@ mod tests { "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " MemoryExec: partitions=1, partition_sizes=[0]" ]); + // Order requirement of the `BoundedWindowAggExec` is not satisfied. We expect to receive error during sanity check. assert_sanity_check(&bw, false); Ok(()) } @@ -501,6 +507,7 @@ mod tests { " MemoryExec: partitions=1, partition_sizes=[0]", ], ); + // Distribution requirement of the `GlobalLimitExec` is not satisfied. We expect to receive error during sanity check. assert_sanity_check(&limit, false); Ok(()) } @@ -609,6 +616,7 @@ mod tests { " MemoryExec: partitions=1, partition_sizes=[0]", ], ); + // Order requirement for the `SortMergeJoin` is not satisfied for right child. We expect to receive error during sanity check. assert_sanity_check(&smj, false); Ok(()) } @@ -655,6 +663,7 @@ mod tests { " MemoryExec: partitions=1, partition_sizes=[0]", ], ); + // Distribution requirement for the `SortMergeJoin` is not satisfied for right child (has round-robin partitioning). We expect to receive error during sanity check. assert_sanity_check(&smj, false); Ok(()) } From e93cf6e629ead66d95ca1c883813e04d179a74f3 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 26 Jun 2024 11:36:14 +0300 Subject: [PATCH 20/28] Update output_requirements.rs --- datafusion/core/src/physical_optimizer/output_requirements.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 89a7b690a82f..4a43b00b2118 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -248,6 +248,8 @@ fn require_top_ordering_helper( if children.len() != 1 { Ok((plan, false)) } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { + // In the case of constant columns, output ordering of SortExec would give an emtpy set. + // Therefore; we check the sort expression field of the SortExec to assign the requirements. let req_ordering = sort_exec.expr(); let req_dist = sort_exec.required_input_distribution()[0].clone(); let reqs = PhysicalSortRequirement::from_sort_exprs(req_ordering); From 58cbf9a091bf2e53fdbed8574b23d120956553d9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 27 Jun 2024 17:53:34 +0300 Subject: [PATCH 21/28] Address reviews --- .../physical_optimizer/output_requirements.rs | 2 +- .../physical-expr/src/equivalence/ordering.rs | 3 +-- .../src/equivalence/properties.rs | 23 +++++++++++-------- datafusion/physical-expr/src/lib.rs | 3 +-- .../physical-plan/src/repartition/mod.rs | 4 ++-- datafusion/physical-plan/src/union.rs | 3 +-- datafusion/physical-plan/src/windows/mod.rs | 3 +-- 7 files changed, 21 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 4a43b00b2118..671bb437d5fa 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -248,7 +248,7 @@ fn require_top_ordering_helper( if children.len() != 1 { Ok((plan, false)) } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - // In the case of constant columns, output ordering of SortExec would give an emtpy set. + // In case of constant columns, output ordering of SortExec would give an empty set. // Therefore; we check the sort expression field of the SortExec to assign the requirements. let req_ordering = sort_exec.expr(); let req_dist = sort_exec.required_input_distribution()[0].clone(); diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 7899e6334c8d..ac9d64e486ac 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -556,8 +556,7 @@ mod tests { let constants = constants .into_iter() - .map(|expr| ConstExpr::new(expr.clone()).with_across_partitions(true)) - .collect::>(); + .map(|expr| ConstExpr::new(expr.clone()).with_across_partitions(true)); eq_properties = eq_properties.add_constants(constants); let reqs = convert_to_sort_exprs(&reqs); diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 2e07fa6315c1..0a5279c0a339 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -19,6 +19,7 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; use super::ordering::collapse_lex_ordering; +use crate::equivalence::class::const_exprs_contains; use crate::equivalence::{ collapse_lex_req, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, }; @@ -38,7 +39,6 @@ use datafusion_physical_expr_common::expressions::column::Column; use datafusion_physical_expr_common::expressions::CastExpr; use datafusion_physical_expr_common::utils::ExprPropertiesNode; -use crate::equivalence::class::const_exprs_contains; use indexmap::{IndexMap, IndexSet}; use itertools::Itertools; @@ -175,9 +175,9 @@ impl EquivalenceProperties { } /// Removes constant expressions that may change across partitions. - /// (Should be used when different partitions are fused) + /// This method should be used when different partitions are fused. pub fn clear_per_partition_constants(&mut self) { - self.constants.retain(|elem| elem.across_partitions()); + self.constants.retain(|item| item.across_partitions()); } /// Extends this `EquivalenceProperties` by adding the orderings inside the @@ -357,8 +357,7 @@ impl EquivalenceProperties { constant_exprs.extend( self.constants .iter() - .map(|const_expr| const_expr.expr().clone()) - .collect::>(), + .map(|const_expr| const_expr.expr().clone()), ); let constants_normalized = self.eq_group.normalize_exprs(constant_exprs); // Prune redundant sections in the requirement: @@ -1352,10 +1351,16 @@ pub fn join_equivalence_properties( on, )); - let left_constants = left.constants().to_vec(); - let right_constants = right.constants().to_vec(); - let left_oeq_class = left.oeq_class; - let mut right_oeq_class = right.oeq_class; + let EquivalenceProperties { + constants: left_constants, + oeq_class: left_oeq_class, + .. + } = left; + let EquivalenceProperties { + constants: right_constants, + oeq_class: mut right_oeq_class, + .. + } = right; match maintains_input_order { [true, false] => { // In this special case, right side ordering can be prefixed with diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index cfc5213d1b76..06c73636773e 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -46,8 +46,7 @@ pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use datafusion_physical_expr_common::aggregate::{ AggregateExpr, AggregatePhysicalExpressions, }; -pub use equivalence::ConstExpr; -pub use equivalence::EquivalenceProperties; +pub use equivalence::{ConstExpr, EquivalenceProperties}; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index a8029132682c..d9e16c98eee8 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -701,8 +701,8 @@ impl RepartitionExec { if !Self::maintains_input_order_helper(input, preserve_order)[0] { eq_properties.clear_orderings(); } - // When input partition is larger than 1, partitions will be fused at the output. - // Remove per partition constants. + // When there are more than one input partitions, they will be fused at the output. + // Therefore, remove per partition constants. if input.output_partitioning().partition_count() > 1 { eq_properties.clear_per_partition_constants(); } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index ffd65adb8ae4..4697e623fdae 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -196,8 +196,7 @@ fn calculate_union_eq_properties( }) .collect::>(); } - eq_properties = eq_properties.add_constants(meet_constants); - eq_properties + eq_properties.add_constants(meet_constants) } impl DisplayAs for UnionExec { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 4710303cfa41..34b1c4069515 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -575,8 +575,7 @@ pub fn get_window_mode( // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs .iter() - .map(|expr| ConstExpr::new(expr.clone())) - .collect::>(); + .map(|expr| ConstExpr::new(expr.clone())); let partition_by_eqs = input_eqs.add_constants(const_exprs); let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); let reverse_order_by_reqs = From 5a34ae0fde7a956a3aa3f6c4713f95d6e1da4150 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Mon, 1 Jul 2024 13:29:57 +0300 Subject: [PATCH 22/28] Update datafusion/core/src/physical_optimizer/optimizer.rs Co-authored-by: Mehmet Ozan Kabak --- datafusion/core/src/physical_optimizer/optimizer.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index a4447934daa5..2d9744ad23dd 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -124,12 +124,12 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), - // The SanityCheckPlan will reject non-runnable query + // The SanityCheckPlan rule checks whether the order and + // distribution requirements of each node in the plan + // is satisfied. It will also reject non-runnable query // plans that use pipeline-breaking operators on infinite - // input(s) as well as checks whether the order and - // distribution requirements of each children of a plan - // are satisfied. The rule generates a diagnostic error - // message when this happens. It makes no changes to the + // input(s). The rule generates a diagnostic error + // message for invalid plans. It makes no changes to the // given query plan; i.e. it only acts as a final // gatekeeping rule. Arc::new(SanityCheckPlan::new()), From f3ac546cbb370c1ca631a2b6346932018311723b Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Mon, 1 Jul 2024 13:30:34 +0300 Subject: [PATCH 23/28] Update datafusion/core/src/physical_optimizer/sanity_checker.rs Co-authored-by: Mehmet Ozan Kabak --- datafusion/core/src/physical_optimizer/sanity_checker.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index e40e6bb21383..097cb453fbae 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -37,11 +37,10 @@ use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties}; use itertools::izip; /// The SanityCheckPlan rule rejects the following query plans: -/// i) Plans that use pipeline-breaking operators on infinite input(s), -/// these queries cannot be executed. -/// ii) Plans in which their order and distribution requirements are not -/// satisfied by its children, these queries will most likely yield -/// incorrect results. +/// 1. Invalid plans containing nodes whose order and/or distribution requirements +/// are not satisfied by their children. +/// 2. Plans that use pipeline-breaking operators on infinite input(s), +/// it is impossible to execute such queries. #[derive(Default)] pub struct SanityCheckPlan {} From 0491c137dbe57aea51f04457a1ce7489b4e963e5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 1 Jul 2024 13:34:46 +0300 Subject: [PATCH 24/28] Address reviews --- .../src/physical_optimizer/sanity_checker.rs | 21 ++++++++----------- .../src/equivalence/properties.rs | 3 +-- datafusion/physical-plan/src/union.rs | 3 ++- 3 files changed, 12 insertions(+), 15 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 097cb453fbae..b4702b6ba8cb 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -126,19 +126,16 @@ pub fn check_plan_sanity( plan.required_input_distribution().iter() ) { let child_eq_props = child.equivalence_properties(); - match child_sort_req { - None => (), - Some(child_sort_req) => { - if !child_eq_props.ordering_satisfy_requirement(child_sort_req) { - let child_plan_str = get_plan_string(child); - return plan_err!( - "Child: {:?} does not satisfy parent order requirements: {:?}", - child_plan_str, - child_sort_req - ); - } + if let Some(child_sort_req) = child_sort_req { + if !child_eq_props.ordering_satisfy_requirement(child_sort_req) { + let child_plan_str = get_plan_string(child); + return plan_err!( + "Child: {:?} does not satisfy parent order requirements: {:?}", + child_plan_str, + child_sort_req + ); } - }; + } if !child .output_partitioning() diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 0a5279c0a339..e1f6e632f9b6 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -959,8 +959,7 @@ impl EquivalenceProperties { let const_exprs = self .constants .iter() - .map(|const_expr| const_expr.expr().clone()) - .collect::>(); + .map(|const_expr| const_expr.expr().clone()); let normalized_constants = self.eq_group.normalize_exprs(const_exprs); let normalized_expr = self.eq_group.normalize_expr(expr.clone()); is_constant_recurse(&normalized_constants, &normalized_expr) diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 4697e623fdae..3f88eb4c3732 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -137,7 +137,8 @@ impl UnionExec { PlanProperties::new(eq_properties, output_partitioning, mode) } } - +/// Calculate `EquivalenceProperties` for `UnionExec` from the `EquivalenceProperties` +/// of its children. fn calculate_union_eq_properties( children_eqs: &[&EquivalenceProperties], schema: SchemaRef, From dfd219c61e84b4d3cfbd847914c327200eb4b5a5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 1 Jul 2024 13:39:13 +0300 Subject: [PATCH 25/28] Minor changes --- datafusion/core/src/physical_optimizer/mod.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 858e9278555a..9ad05bf496e5 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -30,7 +30,6 @@ pub mod join_selection; pub mod limited_distinct_aggregation; pub mod optimizer; pub mod output_requirements; -pub mod pipeline_checker; pub mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; From d8ebc14731a3ef61dc9d06b9a2950ba8be212bf2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur <106137913+mustafasrepo@users.noreply.github.com> Date: Wed, 3 Jul 2024 08:57:04 +0300 Subject: [PATCH 26/28] Apply suggestions from code review Co-authored-by: Andrew Lamb --- datafusion/core/src/physical_optimizer/sanity_checker.rs | 2 +- datafusion/physical-expr/src/equivalence/class.rs | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index b4702b6ba8cb..083b42f7400b 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -40,7 +40,7 @@ use itertools::izip; /// 1. Invalid plans containing nodes whose order and/or distribution requirements /// are not satisfied by their children. /// 2. Plans that use pipeline-breaking operators on infinite input(s), -/// it is impossible to execute such queries. +/// it is impossible to execute such queries (they will never generate output nor finish) #[derive(Default)] pub struct SanityCheckPlan {} diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 265b20582575..7d7132461a31 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -29,10 +29,11 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::JoinType; #[derive(Debug, Clone)] -/// A structure representing a constant expression in a physical execution plan. +/// A structure representing a expression known to be constant in a physical execution plan. /// /// The `ConstExpr` struct encapsulates an expression that is constant during the execution -/// of a query. +/// of a query. For example if a predicate like `A = 5` applied earlier in the plan `A` would +/// be known constant /// /// # Fields /// From 747b69bf70f63266a4a84efafddc265d136c5661 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 3 Jul 2024 09:06:17 +0300 Subject: [PATCH 27/28] Update comment --- datafusion/physical-expr/src/equivalence/class.rs | 2 +- datafusion/physical-expr/src/equivalence/properties.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 7d7132461a31..5f19c6767687 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -32,7 +32,7 @@ use datafusion_common::JoinType; /// A structure representing a expression known to be constant in a physical execution plan. /// /// The `ConstExpr` struct encapsulates an expression that is constant during the execution -/// of a query. For example if a predicate like `A = 5` applied earlier in the plan `A` would +/// of a query. For example if a predicate like `A = 5` applied earlier in the plan `A` would /// be known constant /// /// # Fields diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index e1f6e632f9b6..138768c0c9cc 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -175,7 +175,7 @@ impl EquivalenceProperties { } /// Removes constant expressions that may change across partitions. - /// This method should be used when different partitions are fused. + /// This method should be used when data from different partitions are merged. pub fn clear_per_partition_constants(&mut self) { self.constants.retain(|item| item.across_partitions()); } From aa1382b6e88b47a8082bb12bab8c3ec1ad35b255 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 3 Jul 2024 09:43:08 +0300 Subject: [PATCH 28/28] Add map implementation --- datafusion/physical-expr/src/equivalence/class.rs | 11 +++++++++++ .../physical-expr/src/equivalence/properties.rs | 7 +------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 5f19c6767687..6c12acb934be 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -72,6 +72,17 @@ impl ConstExpr { pub fn owned_expr(self) -> Arc { self.expr } + + pub fn map(&self, f: F) -> Option + where + F: Fn(&Arc) -> Option>, + { + let maybe_expr = f(&self.expr); + maybe_expr.map(|expr| Self { + expr, + across_partitions: self.across_partitions, + }) + } } /// Checks whether `expr` is among in the `const_exprs`. diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 138768c0c9cc..e3a2d1c753ca 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -822,12 +822,7 @@ impl EquivalenceProperties { .constants .iter() .flat_map(|const_expr| { - self.eq_group - .project_expr(mapping, const_expr.expr()) - .map(|expr| { - ConstExpr::new(expr) - .with_across_partitions(const_expr.across_partitions()) - }) + const_expr.map(|expr| self.eq_group.project_expr(mapping, expr)) }) .collect::>(); // Add projection expressions that are known to be constant: