From a91e642e81fefd1c4b5c9b3184adb7dc5106522d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 15 Feb 2024 09:37:25 +0300 Subject: [PATCH] Draft PR is moved to new repo --- datafusion/core/src/dataframe/parquet.rs | 2 +- .../combine_partial_final_agg.rs | 6 +- .../enforce_distribution.rs | 6 +- .../limited_distinct_aggregation.rs | 4 +- datafusion/core/src/physical_optimizer/mod.rs | 2 +- .../optimize_projections.rs | 5020 +++++++++++++++++ .../core/src/physical_optimizer/optimizer.rs | 13 +- .../physical_optimizer/projection_pushdown.rs | 2446 -------- .../physical_optimizer/topk_aggregation.rs | 2 +- datafusion/core/tests/sql/explain_analyze.rs | 22 +- datafusion/physical-expr/src/aggregate/mod.rs | 9 + datafusion/physical-expr/src/aggregate/sum.rs | 13 + .../physical-expr/src/window/window_expr.rs | 13 +- .../physical-plan/src/aggregates/mod.rs | 6 +- datafusion/physical-plan/src/lib.rs | 1 + datafusion/physical-plan/src/projection.rs | 28 +- datafusion/sqllogictest/test_files/cte.slt | 3 +- .../sqllogictest/test_files/distinct_on.slt | 2 +- .../sqllogictest/test_files/explain.slt | 10 +- .../join_disable_repartition_joins.slt | 17 +- datafusion/sqllogictest/test_files/limit.slt | 6 +- .../sqllogictest/test_files/predicates.slt | 16 +- datafusion/sqllogictest/test_files/select.slt | 1 + datafusion/sqllogictest/test_files/window.slt | 32 +- 24 files changed, 5152 insertions(+), 2528 deletions(-) create mode 100644 datafusion/core/src/physical_optimizer/optimize_projections.rs delete mode 100644 datafusion/core/src/physical_optimizer/projection_pushdown.rs diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 00a0e780d51f..5cd451422aec 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -125,7 +125,7 @@ mod tests { let plan = df.explain(false, false)?.collect().await?; // Filters all the way to Parquet let formatted = pretty::pretty_format_batches(&plan)?.to_string(); - assert!(formatted.contains("FilterExec: id@0 = 1")); + assert!(formatted.contains("Filter: test.id = Int32(1)")); Ok(()) } diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 61eb2381c63b..93219246830a 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -70,12 +70,12 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { AggregateMode::Partial ) && can_combine( ( - agg_exec.group_by(), + agg_exec.group_expr(), agg_exec.aggr_expr(), agg_exec.filter_expr(), ), ( - input_agg_exec.group_by(), + input_agg_exec.group_expr(), input_agg_exec.aggr_expr(), input_agg_exec.filter_expr(), ), @@ -88,7 +88,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { }; AggregateExec::try_new( mode, - input_agg_exec.group_by().clone(), + input_agg_exec.group_expr().clone(), input_agg_exec.aggr_expr().to_vec(), input_agg_exec.filter_expr().to_vec(), input_agg_exec.input().clone(), diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 4f8806a68592..80418e0edbe7 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -460,7 +460,7 @@ fn reorder_aggregate_keys( ) -> Result { let parent_required = &agg_node.data; let output_columns = agg_exec - .group_by() + .group_expr() .expr() .iter() .enumerate() @@ -473,7 +473,7 @@ fn reorder_aggregate_keys( .collect::>(); if parent_required.len() == output_exprs.len() - && agg_exec.group_by().null_expr().is_empty() + && agg_exec.group_expr().null_expr().is_empty() && !physical_exprs_equal(&output_exprs, parent_required) { if let Some(positions) = expected_expr_positions(&output_exprs, parent_required) { @@ -481,7 +481,7 @@ fn reorder_aggregate_keys( agg_exec.input().as_any().downcast_ref::() { if matches!(agg_exec.mode(), &AggregateMode::Partial) { - let group_exprs = agg_exec.group_by().expr(); + let group_exprs = agg_exec.group_expr().expr(); let new_group_exprs = positions .into_iter() .map(|idx| group_exprs[idx].clone()) diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index 9855247151b8..58d2ba84eeb9 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -52,7 +52,7 @@ impl LimitedDistinctAggregation { // We found what we want: clone, copy the limit down, and return modified node let new_aggr = AggregateExec::try_new( *aggr.mode(), - aggr.group_by().clone(), + aggr.group_expr().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), aggr.input().clone(), @@ -113,7 +113,7 @@ impl LimitedDistinctAggregation { if let Some(parent_aggr) = match_aggr.as_any().downcast_ref::() { - if !parent_aggr.group_by().eq(aggr.group_by()) { + if !parent_aggr.group_expr().eq(aggr.group_expr()) { // a partial and final aggregation with different groupings disqualifies // rewriting the child aggregation rewrite_applicable = false; diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index e990fead610d..c9931b3f06f8 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -28,10 +28,10 @@ pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; pub mod limited_distinct_aggregation; +mod optimize_projections; pub mod optimizer; pub mod output_requirements; pub mod pipeline_checker; -mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; mod sort_pushdown; diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs new file mode 100644 index 000000000000..3e968db99281 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -0,0 +1,5020 @@ +// 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. + +//! OptimizeProjections rule aims achieving the most effective use of projections +//! in plans. It ensures that query plans are free from unnecessary projections +//! and that no unused columns are propagated unnecessarily between plans. +//! +//! The rule is designed to enhance query performance by: +//! 1. Preventing the transfer of unused columns from leaves to root. +//! 2. Ensuring projections are used only when they contribute to narrowing the schema, +//! or when necessary for evaluation or aliasing. +//! +//! The optimization is conducted in two phases: +//! +//! Top-down Phase: +//! --------------- +//! - Traverses the plan from root to leaves. If the node is: +//! 1. Projection node, it may: +//! a) Merge it with its input projection if merge is beneficial. +//! b) Remove the projection if it is redundant. +//! c) Narrow the Projection if possible. +//! d) The projection can be nested into the source. +//! e) Do nothing, otherwise. +//! 2. Non-Projection node: +//! a) Schema needs pruning. Insert the necessary projections to the children. +//! b) All fields are required. Do nothing. +//! +//! Bottom-up Phase (now resides in map_children() implementation): +//! ---------------- +//! This pass is required because modifying a plan node can change the column +//! indices used by output nodes. When such a change occurs, we store the old +//! and new indices of the columns in the node's state. We then proceed from +//! the leaves to the root, updating the indices of columns in the plans by +//! referencing these mapping records. After the top-down phase, also some +//! unnecessary projections may emerge. When projections check its input schema +//! mapping, it can remove itself and assign new schema mapping to the new node +//! which was the projection's input formerly. + +use std::collections::{HashMap, HashSet}; +use std::mem; +use std::sync::Arc; + +use super::PhysicalOptimizerRule; +use crate::datasource::physical_plan::CsvExec; +use crate::error::Result; +use crate::physical_plan::filter::FilterExec; +use crate::physical_plan::projection::ProjectionExec; +use crate::physical_plan::ExecutionPlan; + +use arrow_schema::SchemaRef; +use chrono::naive; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::DataFusionError; +use datafusion_common::{internal_err, JoinSide, JoinType}; +use datafusion_physical_expr::expressions::{Column, Literal}; +use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::{Partitioning, PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::insert::FileSinkExec; +use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter, JoinOn}; +use datafusion_physical_plan::joins::{ + CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, + SymmetricHashJoinExec, +}; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::union::{InterleaveExec, UnionExec}; +use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; +use datafusion_physical_plan::{displayable, get_plan_string}; +use itertools::{Interleave, Itertools}; + +/// The tree node for the rule of [`OptimizeProjections`]. It stores the necessary +/// fields for column requirements and changed indices of columns. +#[derive(Debug, Clone)] +pub struct ProjectionOptimizer { + pub plan: Arc, + /// The node above expects it can reach these columns. + /// Note: This set can be built on column indices rather than column expressions. + pub required_columns: HashSet, + /// The nodes above will be updated according to these mathces. First element indicates + /// the initial column index, and the second element is for the updated version. + pub schema_mapping: HashMap, + pub children_nodes: Vec, +} + +/// This type defines whether a column is required, in case of pairing with `true` value, or is +/// not required, in case of pairing with `false`. It is constructed based on output schema of a plan. +type ColumnRequirements = HashMap; + +impl ProjectionOptimizer { + /// Constructs the empty graph according to the plan. All state information is empty initially. + fn new_default(plan: Arc) -> Self { + let children = plan.children(); + Self { + plan, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: children.into_iter().map(Self::new_default).collect(), + } + } + + /// Recursively called transform function while traversing from root node + /// to leaf nodes. It only addresses the self and child node, and make + /// the necessary changes on them, does not deep dive. + fn adjust_node_with_requirements(mut self) -> Result { + // print_plan(&self.plan); + // println!("self reqs: {:?}", self.required_columns); + // println!("self map: {:?}", self.schema_mapping); + // self.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); + + // If the node is a source provdider, no need a change. + if self.children_nodes.len() == 0 { + return Ok(self); + } + + if self.plan.as_any().is::() { + // If the node is a projection, it is analyzed and may be rewritten + // in a most effective way, or even removed. + self.optimize_projections() + } else { + // If the node corresponds to any other plan, a projection may be inserted to its input. + self.try_projection_insertion() + } + } + + /// The function tries 4 cases: + /// 1) If the input plan is also a projection, they can be merged into one projection. + /// 2) The projection can be removed. + /// 3) The projection can get narrower. + /// 4) The projection can be embedded into the source. + /// If none of them is possible, it remains unchanged. + pub fn optimize_projections(mut self) -> Result { + let projection_input = self.plan.children(); + let projection_input = projection_input[0].as_any(); + + // We first need to check having 2 sequential projections in case of merging them. + if projection_input.is::() { + self = match self.try_unifying_projections()? { + Transformed::Yes(unified_plans) => { + // We need to re-run the rule on the new node since it may need further optimizations. + // There may be 3 sequential projections, or the unified node may also be removed or narrowed. + return unified_plans.optimize_projections(); + } + Transformed::No(no_change) => no_change, + }; + } + + // The projection can be removed. To avoid making unnecessary operations, + // try_remove should be called before try_narrow. + self = match self.try_remove_projection() { + Transformed::Yes(removed) => { + // We need to re-run the rule on the current node. It is + // a new plan node and may need optimizations for sure. + return removed.adjust_node_with_requirements(); + } + Transformed::No(no_change) => no_change, + }; + + // The projection can get narrower. + self = match self.try_narrow_projection()? { + Transformed::Yes(narrowed) => { + return Ok(narrowed); + } + Transformed::No(no_change) => no_change, + }; + + // Source providers: + if projection_input.is::() { + self = match self.try_projected_csv() { + Transformed::Yes(new_csv) => return Ok(new_csv), + Transformed::No(no_change) => no_change, + } + } + + // If none of them possible, we will continue to next node. Output requirements + // of the projection in terms of projection input are inserted to child node. + let Some(projection_plan) = self.plan.as_any().downcast_ref::() + else { + return internal_err!( + "\"optimize_projections\" subrule must be used on ProjectionExec's." + ); + }; + // If there is nothing that could be better, insert the child requirements and continue. + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .flat_map(|e| collect_columns(&projection_plan.expr()[e.index()].0)) + .collect::>(); + Ok(self) + } + + /// Unifies `projection` with its input, which is also a [`ProjectionExec`], if it is beneficial. + fn try_unifying_projections(mut self) -> Result> { + // These are known to be a ProjectionExec. + let projection = self.plan.as_any().downcast_ref::().unwrap(); + let child_projection = self.children_nodes[0] + .plan + .as_any() + .downcast_ref::() + .unwrap(); + + if caching_projections(projection, child_projection) { + return Ok(Transformed::No(self)); + } + + let mut projected_exprs = vec![]; + for (expr, alias) in projection.expr() { + let Some(expr) = update_expr(expr, child_projection.expr(), true)? else { + return Ok(Transformed::No(self)); + }; + projected_exprs.push((expr, alias.clone())); + } + + let new_plan = + ProjectionExec::try_new(projected_exprs, child_projection.input().clone()) + .map(|e| Arc::new(e) as _)?; + Ok(Transformed::Yes(ProjectionOptimizer { + plan: new_plan, + // Schema of the projection does not change, + // so no need any update on state variables. + required_columns: self.required_columns, + schema_mapping: self.schema_mapping, + children_nodes: self.children_nodes.swap_remove(0).children_nodes, + })) + } + + /// Tries to remove the [`ProjectionExec`]. When these conditions are satisfied, + /// the projection can be safely removed: + /// 1) Projection must have all column expressions without aliases. + /// 2) Projection input is fully required by the projection output requirements. + fn try_remove_projection(mut self) -> Transformed { + // It must be a projection + let projection_exec = + self.plan.as_any().downcast_ref::().unwrap(); + + // The projection must have all column expressions without aliases. + if !all_alias_free_columns(projection_exec.expr()) { + return Transformed::No(self); + } + // The expressions are known to be all columns. + let projection_columns = projection_exec + .expr() + .iter() + .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) + .cloned() + .collect::>(); + + // Input requirements of the projection in terms of projection's parent requirements: + let projection_requires = self + .required_columns + .iter() + .map(|column| projection_columns[column.index()].clone()) + .collect::>(); + + // If all fields of the input are necessary, we can remove the projection. + let input_columns = collect_columns_in_plan_schema(projection_exec.input()); + if input_columns + .iter() + .all(|input_column| projection_requires.contains(&input_column)) + { + let new_mapping = self + .required_columns + .into_iter() + .filter_map(|column| { + let col_ind = column.index(); + if column != projection_columns[col_ind] { + Some((column, projection_columns[col_ind].clone())) + } else { + None + } + }) + .collect(); + + let replaced_child = self.children_nodes.swap_remove(0); + Transformed::Yes(ProjectionOptimizer { + plan: replaced_child.plan, + required_columns: projection_requires, + schema_mapping: new_mapping, + children_nodes: replaced_child.children_nodes, + }) + } else { + Transformed::No(self) + } + } + + /// Compares the inputs and outputs of the projection. If the projection can be + /// rewritten with a narrower schema, it is done so. Otherwise, it returns `None`. + fn try_narrow_projection(self) -> Result> { + // It must be a projection. + let projection_exec = + self.plan.as_any().downcast_ref::().unwrap(); + + // Check for the projection output if it has any redundant elements. + let projection_output_columns = projection_exec + .expr() + .iter() + .enumerate() + .map(|(i, (_e, a))| Column::new(a, i)) + .collect::>(); + let used_indices = projection_output_columns + .iter() + .filter(|&p_out| self.required_columns.contains(p_out)) + .map(|p_out| p_out.index()) + .collect::>(); + + if used_indices.len() == projection_output_columns.len() { + // All projected items are used. + return Ok(Transformed::No(self)); + } + + // New projected expressions are rewritten according to used indices. + let new_projection = used_indices + .iter() + .map(|i| projection_exec.expr()[*i].clone()) + .collect::>(); + + // Construct the mapping. + let mut schema_mapping = HashMap::new(); + for (new_idx, old_idx) in used_indices.iter().enumerate() { + if new_idx != *old_idx { + schema_mapping.insert( + projection_output_columns[*old_idx].clone(), + projection_output_columns[new_idx].clone(), + ); + } + } + + let new_projection_plan = Arc::new(ProjectionExec::try_new( + new_projection.clone(), + self.children_nodes[0].plan.clone(), + )?); + let new_projection_requires = self + .required_columns + .iter() + .map(|col| schema_mapping.get(col).cloned().unwrap_or(col.clone())) + .collect(); + let mut new_node = ProjectionOptimizer { + plan: new_projection_plan, + required_columns: new_projection_requires, + schema_mapping, + children_nodes: self.children_nodes, + }; + + // Since the rule work on the child node now, we need to insert child note requirements here. + new_node.children_nodes[0].required_columns = self + .required_columns + .iter() + .flat_map(|column| collect_columns(&new_projection[column.index()].0)) + .collect::>(); + + Ok(Transformed::Yes(new_node)) + } + + /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. + fn try_projected_csv(self) -> Transformed { + // These plans are known. + let projection = self.plan.as_any().downcast_ref::().unwrap(); + let csv = projection + .input() + .as_any() + .downcast_ref::() + .unwrap(); + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into CsvExec, but it could be a conflict of their responsibility. + if all_alias_free_columns(projection.expr()) { + let mut file_scan = csv.base_config().clone(); + let projection_columns = projection + .expr() + .iter() + .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) + .collect::>(); + let new_projections = + new_projections_for_columns(&projection_columns, &file_scan.projection); + + file_scan.projection = Some(new_projections); + + Transformed::Yes(ProjectionOptimizer { + plan: Arc::new(CsvExec::new( + file_scan, + csv.has_header(), + csv.delimiter(), + csv.quote(), + csv.escape(), + csv.file_compression_type, + )) as _, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), // Sources cannot have a mapping. + children_nodes: vec![], + }) + } else { + Transformed::No(self) + } + } + + /// If the node plan can be rewritten with a narrower schema, a projection is inserted + /// into its input to do so. The node plans are rewritten according to its new input, + /// and the mapping of old indices vs. new indices is put to node's related field. + /// When this function returns and recursion on the node finishes, the upper node plans + /// are rewritten according to this mapping. This function also updates the parent + /// requirements and extends them with self requirements before inserting them to its child(ren). + fn try_projection_insertion(mut self) -> Result { + let plan = self.plan.clone(); + + if let Some(_projection) = plan.as_any().downcast_ref::() { + panic!( + "\"try_projection_insertion\" subrule cannot be used on ProjectionExec's." + ); + } else if let Some(_csv) = plan.as_any().downcast_ref::() { + panic!("\"try_projection_insertion\" subrule cannot be used on plans with no child.") + } + // These plans preserve the input schema, and do not add new requirements. + else if let Some(coal_b) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_coalesce_batches(coal_b)?; + } else if let Some(_) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_coalesce_partitions()?; + } else if let Some(glimit) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_global_limit(glimit)?; + } else if let Some(llimit) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_local_limit(llimit)?; + } + // These plans also preserve the input schema, but may extend requirements. + else if let Some(filter) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_filter(filter)?; + } else if let Some(repartition) = plan.as_any().downcast_ref::() + { + self = self.try_insert_below_repartition(repartition)?; + } else if let Some(sort) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_sort(sort)?; + } else if let Some(sortp_merge) = + plan.as_any().downcast_ref::() + { + self = self.try_insert_below_sort_preserving_merge(sortp_merge)?; + } + // Preserves schema and do not change requirements, but have multi-child. + else if let Some(_) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_union()?; + } else if let Some(_) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_interleave()?; + } + // Concatenates schemas and do not change requirements. + else if let Some(cj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_cross_join(cj)? + } + // Specially handled joins and aggregations + else if let Some(hj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_hash_join(hj)? + } else if let Some(nlj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_nested_loop_join(nlj)? + } else if let Some(smj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_sort_merge_join(smj)? + } else if let Some(shj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_symmetric_hash_join(shj)? + } else if let Some(agg) = plan.as_any().downcast_ref::() { + if agg.aggr_expr().iter().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .is_none() + }) { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + self = self.try_insert_below_aggregate(agg)? + } else if let Some(w_agg) = plan.as_any().downcast_ref::() { + if w_agg.window_expr().iter().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .is_none() + }) { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + self = self.try_insert_below_window_aggregate(w_agg)? + } else if let Some(bw_agg) = plan.as_any().downcast_ref::() + { + if bw_agg.window_expr().iter().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .is_none() + }) { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + self = self.try_insert_below_bounded_window_aggregate(bw_agg)? + } else if let Some(file_sink) = plan.as_any().downcast_ref::() { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan) + } else { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + Ok(self) + } + + fn try_insert_below_coalesce_batches( + mut self, + coal_batches: &CoalesceBatchesExec, + ) -> Result { + // CoalesceBatchesExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = Arc::new(CoalesceBatchesExec::new( + new_child.plan.clone(), + coal_batches.target_batch_size(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_coalesce_partitions(mut self) -> Result { + // CoalescePartitionsExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = Arc::new(CoalescePartitionsExec::new(new_child.plan.clone())) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_global_limit( + mut self, + glimit: &GlobalLimitExec, + ) -> Result { + // GlobalLimitExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if true { + // if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = Arc::new(GlobalLimitExec::new( + new_child.plan.clone(), + glimit.skip(), + glimit.fetch(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_local_limit( + mut self, + llimit: &LocalLimitExec, + ) -> Result { + // LocalLimitExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = + Arc::new(LocalLimitExec::new(new_child.plan.clone(), llimit.fetch())) + as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_filter( + mut self, + filter: &FilterExec, + ) -> Result { + // FilterExec extends the requirements with the columns in its predicate. + self.required_columns + .extend(collect_columns(filter.predicate())); + + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + // Rewrite the predicate with possibly updated column indices. + let new_predicate = update_column_index(filter.predicate(), &schema_mapping); + let plan = + Arc::new(FilterExec::try_new(new_predicate, new_child.plan.clone())?) + as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_repartition( + mut self, + repartition: &RepartitionExec, + ) -> Result { + // If RepartitionExec applies a hash repartition, it extends + // the requirements with the columns in the hashed expressions. + if let Partitioning::Hash(exprs, _size) = repartition.partitioning() { + self.required_columns + .extend(exprs.iter().flat_map(|expr| collect_columns(expr))); + } + + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + // Rewrite the hashed expressions if there is any with possibly updated column indices. + let new_partitioning = + if let Partitioning::Hash(exprs, size) = repartition.partitioning() { + Partitioning::Hash( + exprs + .iter() + .map(|expr| update_column_index(expr, &schema_mapping)) + .collect::>(), + *size, + ) + } else { + repartition.partitioning().clone() + }; + let plan = Arc::new(RepartitionExec::try_new( + new_child.plan.clone(), + new_partitioning, + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_sort(mut self, sort: &SortExec) -> Result { + // SortExec extends the requirements with the columns in its sort expressions. + self.required_columns.extend( + sort.expr() + .iter() + .flat_map(|sort_expr| collect_columns(&sort_expr.expr)), + ); + + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + // Rewrite the sort expressions with possibly updated column indices. + let new_sort_exprs = sort + .expr() + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, &schema_mapping), + options: sort_expr.options, + }) + .collect::>(); + let plan = + Arc::new(SortExec::new(new_sort_exprs, new_child.plan.clone())) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_sort_preserving_merge( + mut self, + sortp_merge: &SortPreservingMergeExec, + ) -> Result { + // SortPreservingMergeExec extends the requirements with the columns in its sort expressions. + self.required_columns.extend( + sortp_merge + .expr() + .iter() + .flat_map(|sort_expr| collect_columns(&sort_expr.expr)), + ); + + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + // Rewrite the sort expressions with possibly updated column indices. + let new_sort_exprs = sortp_merge + .expr() + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, &schema_mapping), + options: sort_expr.options, + }) + .collect::>(); + let plan = Arc::new(SortPreservingMergeExec::new( + new_sort_exprs, + new_child.plan.clone(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_union(mut self) -> Result { + // UnionExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + let required_columns = mem::take(&mut self.required_columns); + self.children_nodes + .iter_mut() + .for_each(|c| c.required_columns = required_columns.clone()); + } else { + let (new_children, schema_mapping) = + self.insert_multi_projection_below_union(requirement_map)?; + let plan = Arc::new(UnionExec::new( + new_children.iter().map(|c| c.plan.clone()).collect(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: new_children, + } + } + Ok(self) + } + + fn try_insert_below_interleave(mut self) -> Result { + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + let required_columns = mem::take(&mut self.required_columns); + self.children_nodes + .iter_mut() + .for_each(|c| c.required_columns = required_columns.clone()); + } else { + let (new_children, schema_mapping) = + self.insert_multi_projection_below_union(requirement_map)?; + let plan = Arc::new(InterleaveExec::try_new( + new_children.iter().map(|c| c.plan.clone()).collect(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: new_children, + } + } + Ok(self) + } + + fn try_insert_below_cross_join( + mut self, + cj: &CrossJoinExec, + ) -> Result { + let left_size = cj.left().schema().fields().len(); + // CrossJoinExec does not add new requirements. + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (true, true) => { + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + let plan = Arc::new(CrossJoinExec::new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + // Left child needs a projection. + (true, false) => { + let right_child = self.children_nodes.swap_remove(1); + let (new_left_child, left_schema_mapping) = + self.insert_projection_below_single_child(analyzed_join_left, 0)?; + let plan = Arc::new(CrossJoinExec::new( + new_left_child.plan.clone(), + right_child.plan.clone(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + // Right child needs a projection. + (false, true) => { + let left_child = self.children_nodes[0].clone(); + let (new_right_child, mut right_schema_mapping) = + self.insert_projection_below_single_child(analyzed_join_right, 1)?; + right_schema_mapping = right_schema_mapping + .into_iter() + .map(|(old, new)| { + ( + Column::new(old.name(), old.index() + left_size), + Column::new(new.name(), new.index() + left_size), + ) + }) + .collect(); + let plan = Arc::new(CrossJoinExec::new( + left_child.plan.clone(), + new_right_child.plan.clone(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (false, false) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + Ok(self) + } + + fn try_insert_below_hash_join( + mut self, + hj: &HashJoinExec, + ) -> Result { + let left_size = hj.left().schema().fields().len(); + // HashJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. + match hj.join_type() { + JoinType::RightAnti | JoinType::RightSemi => { + self.required_columns = self + .required_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect() + } + _ => {} + } + self.required_columns + .extend(collect_columns_in_join_conditions( + hj.on(), + hj.filter(), + left_size, + self.children_nodes[0].plan.schema(), + self.children_nodes[1].plan.schema(), + )); + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + + match hj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (false, false) => { + let new_on = update_equivalence_conditions( + hj.on(), + &analyzed_join_left, + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &analyzed_join_left, + &analyzed_join_right, + ); + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + let plan = Arc::new(HashJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + hj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &analyzed_join_right, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(HashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let left_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + hj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(HashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + match all_columns_required(&analyzed_join_left) { + false => { + let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + hj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &analyzed_join_left, + &HashMap::new(), + ); + + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(HashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + right_child.required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + true => { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes[1].required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect() + } + } + } + JoinType::RightAnti | JoinType::RightSemi => { + match all_columns_required(&analyzed_join_right) { + false => { + let mut left_child = self.children_nodes.swap_remove(0); + let new_on = update_equivalence_conditions( + hj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(HashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + left_child.required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + true => { + self.children_nodes[0].required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self.children_nodes[1].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[1].plan); + } + } + } + } + Ok(self) + } + + fn try_insert_below_nested_loop_join( + mut self, + nlj: &NestedLoopJoinExec, + ) -> Result { + let left_size = nlj.left().schema().fields().len(); + // NestedLoopJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. + self.required_columns + .extend(collect_columns_in_join_conditions( + &[], + nlj.filter(), + left_size, + self.children_nodes[0].plan.schema(), + self.children_nodes[1].plan.schema(), + )); + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + + match nlj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (false, false) => { + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &analyzed_join_left, + &analyzed_join_right, + ); + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let right_child = self.children_nodes.swap_remove(1); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &analyzed_join_right, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let left_child = self.children_nodes.swap_remove(1); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + match all_columns_required(&analyzed_join_left) { + false => { + let mut right_child = self.children_nodes.swap_remove(1); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &analyzed_join_left, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + right_child.required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + true => { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes[1].required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect() + } + } + } + JoinType::RightAnti | JoinType::RightSemi => { + match all_columns_required(&analyzed_join_right) { + false => { + let mut left_child = self.children_nodes.swap_remove(0); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + left_child.required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + true => { + self.children_nodes[0].required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self.children_nodes[1].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[1].plan); + } + } + } + } + Ok(self) + } + + fn try_insert_below_sort_merge_join( + mut self, + smj: &SortMergeJoinExec, + ) -> Result { + let left_size = smj.left().schema().fields().len(); + // SortMergeJoin extends the requirements with the columns in its equivalence and non-equivalence conditions. + self.required_columns + .extend(collect_columns_in_join_conditions( + smj.on(), + None, + left_size, + self.children_nodes[0].plan.schema(), + self.children_nodes[1].plan.schema(), + )); + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + + match smj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (false, false) => { + let new_on = update_equivalence_conditions( + smj.on(), + &analyzed_join_left, + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &analyzed_join_left, + &analyzed_join_right, + ); + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + smj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &analyzed_join_right, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let left_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + smj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + match all_columns_required(&analyzed_join_left) { + false => { + let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + smj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &analyzed_join_left, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + right_child.required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + true => { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes[1].required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect() + } + } + } + JoinType::RightAnti | JoinType::RightSemi => { + match all_columns_required(&analyzed_join_right) { + false => { + let mut left_child = self.children_nodes.swap_remove(0); + let new_on = update_equivalence_conditions( + smj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + left_child.required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + true => { + self.children_nodes[0].required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self.children_nodes[1].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[1].plan); + } + } + } + } + Ok(self) + } + + fn try_insert_below_symmetric_hash_join( + mut self, + shj: &SymmetricHashJoinExec, + ) -> Result { + let left_size = shj.left().schema().fields().len(); + // SymmetricHashJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. + self.required_columns + .extend(collect_columns_in_join_conditions( + shj.on(), + shj.filter(), + left_size, + self.children_nodes[0].plan.schema(), + self.children_nodes[1].plan.schema(), + )); + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + + match shj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (false, false) => { + let new_on = update_equivalence_conditions( + shj.on(), + &analyzed_join_left, + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &analyzed_join_left, + &analyzed_join_right, + ); + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + + let plan = Arc::new(SymmetricHashJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + // TODO: update these + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + shj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &analyzed_join_right, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(SymmetricHashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let left_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + shj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(SymmetricHashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + match all_columns_required(&analyzed_join_left) { + false => { + let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + shj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &analyzed_join_left, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(SymmetricHashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + right_child.required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + true => { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes[1].required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect() + } + } + } + JoinType::RightAnti | JoinType::RightSemi => { + match all_columns_required(&analyzed_join_right) { + false => { + let mut left_child = self.children_nodes.swap_remove(0); + let new_on = update_equivalence_conditions( + shj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(SymmetricHashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + left_child.required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + true => { + self.children_nodes[0].required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self.children_nodes[1].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[1].plan); + } + } + } + } + Ok(self) + } + + fn try_insert_below_aggregate( + mut self, + agg: &AggregateExec, + ) -> Result { + // `AggregateExec` applies their own projections. We can only limit + // the aggregate expressions unless they are used in the upper plans. + let group_columns_len = agg.group_expr().expr().len(); + let required_indices = self + .required_columns + .iter() + .map(|req_col| req_col.index()) + .collect::>(); + let unused_aggr_exprs = agg + .aggr_expr() + .iter() + .enumerate() + .filter(|(idx, _expr)| !required_indices.contains(&(idx + group_columns_len))) + .map(|(idx, _expr)| idx) + .collect::>(); + + if !unused_aggr_exprs.is_empty() { + let new_plan = AggregateExec::try_new( + agg.mode().clone(), + agg.group_expr().clone(), + agg.aggr_expr() + .iter() + .enumerate() + .filter(|(idx, _expr)| !unused_aggr_exprs.contains(idx)) + .map(|(_idx, expr)| expr.clone()) + .collect(), + agg.filter_expr().to_vec(), + agg.input().clone(), + agg.input_schema(), + )?; + self.children_nodes[0].required_columns = new_plan + .group_expr() + .expr() + .iter() + .map(|(e, alias)| collect_columns(e)) + .flatten() + .collect(); + self.children_nodes[0].required_columns.extend( + new_plan + .aggr_expr() + .iter() + .map(|e| { + e.expressions() + .iter() + .map(|e| collect_columns(e)) + .flatten() + .collect::>() + }) + .flatten(), + ); + self.plan = Arc::new(new_plan); + self.required_columns = HashSet::new(); + } else { + match agg.mode() { + datafusion_physical_plan::aggregates::AggregateMode::Final + | datafusion_physical_plan::aggregates::AggregateMode::FinalPartitioned => + { + let mut group_expr_len = agg.group_expr().expr().iter().count(); + let aggr_columns = agg + .aggr_expr() + .iter() + .flat_map(|e| { + e.state_fields() + .unwrap() + .iter() + .map(|field| { + group_expr_len += 1; + Column::new(field.name(), group_expr_len - 1) + }) + .collect::>() + }) + .collect::>(); + let group_columns = agg + .group_expr() + .expr() + .iter() + .flat_map(|(expr, _name)| collect_columns(expr)) + .collect::>(); + let filter_columns = agg + .filter_expr() + .iter() + .filter_map(|expr| expr.as_ref().map(collect_columns)) + .flatten() + .collect::>(); + self.children_nodes[0].required_columns.extend( + aggr_columns + .into_iter() + .chain(group_columns.into_iter()) + .chain(filter_columns.into_iter()), + ) + } + _ => { + let aggr_columns = agg + .aggr_expr() + .iter() + .flat_map(|e| { + e.expressions() + .iter() + .flat_map(collect_columns) + .collect::>() + }) + .collect::>(); + let group_columns = agg + .group_expr() + .expr() + .iter() + .flat_map(|(expr, _name)| collect_columns(expr)) + .collect::>(); + let filter_columns = agg + .filter_expr() + .iter() + .filter_map(|expr| expr.as_ref().map(collect_columns)) + .flatten() + .collect::>(); + self.children_nodes[0].required_columns.extend( + aggr_columns + .into_iter() + .chain(group_columns.into_iter()) + .chain(filter_columns.into_iter()), + ); + } + }; + } + Ok(self) + } + + fn try_insert_below_window_aggregate( + mut self, + w_agg: &WindowAggExec, + ) -> Result { + // Both tries to insert a projection to narrow input columns, and tries to narrow the window + // expressions. If none of them survives, we can even remove the window execution plan. + self.required_columns + .extend(w_agg.window_expr().iter().flat_map(|window_expr| { + window_expr + .expressions() + .iter() + .flat_map(|expr| collect_columns(&expr)) + .collect::>() + })); + self.required_columns.extend( + w_agg + .partition_keys + .iter() + .flat_map(|key| collect_columns(key)), + ); + let requirement_map = self.analyze_requirements(); + if !all_columns_required(&requirement_map) { + if window_agg_required( + w_agg.input().schema().fields().len(), + &requirement_map, + ) { + let (new_child, schema_mapping, window_usage) = self + .clone() + .insert_projection_below_window(w_agg, requirement_map)?; + // Rewrite the sort expressions with possibly updated column indices. + let new_window_exprs = w_agg + .window_expr() + .iter() + .zip(window_usage.clone()) + .filter(|(_window_expr, (_window_col, usage))| *usage) + .map(|(window_expr, (_window_col, _usage))| { + window_expr.clone().with_new_expressions( + window_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, &schema_mapping)) + .collect(), + ) + }) + .collect::>>() + .unwrap(); + + let new_keys = w_agg + .partition_keys + .iter() + .zip(window_usage) + .filter_map(|(key, (_column, usage))| { + if usage { + Some(update_column_index(key, &schema_mapping)) + } else { + None + } + }) + .collect(); + let plan = Arc::new(WindowAggExec::try_new( + new_window_exprs, + new_child.plan.clone(), + new_keys, + )?) as _; + let required_columns = collect_columns_in_plan_schema(&plan); + self = ProjectionOptimizer { + plan, + required_columns, + schema_mapping, + children_nodes: vec![new_child], + } + } else { + // Remove the WindowAggExec + self = self.children_nodes.swap_remove(0); + self.required_columns = requirement_map + .into_iter() + .filter_map(|(column, used)| if used { Some(column) } else { None }) + .collect(); + } + } else { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .filter(|col| { + col.index() + < w_agg.schema().fields().len() - w_agg.window_expr().len() + }) + .cloned() + .collect(); + } + Ok(self) + } + + fn try_insert_below_bounded_window_aggregate( + mut self, + bw_agg: &BoundedWindowAggExec, + ) -> Result { + // Both tries to insert a projection to narrow input columns, and tries to narrow the window + // expressions. If none of them survives, we can even remove the window execution plan. + self.required_columns + .extend(bw_agg.window_expr().iter().flat_map(|window_expr| { + window_expr + .expressions() + .iter() + .flat_map(|expr| collect_columns(&expr)) + .collect::>() + })); + self.required_columns.extend( + bw_agg + .partition_keys + .iter() + .flat_map(|key| collect_columns(key)), + ); + let requirement_map = self.analyze_requirements(); + if !all_columns_required(&requirement_map) { + if window_agg_required( + bw_agg.input().schema().fields().len(), + &requirement_map, + ) { + let (new_child, schema_mapping, window_usage) = self + .clone() + .insert_projection_below_bounded_window(bw_agg, requirement_map)?; + // Rewrite the sort expressions with possibly updated column indices. + let new_window_exprs = bw_agg + .window_expr() + .iter() + .zip(window_usage.clone()) + .filter(|(_window_expr, (_window_col, usage))| *usage) + .map(|(window_expr, (_window_col, _usage))| { + window_expr.clone().with_new_expressions( + window_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, &schema_mapping)) + .collect(), + ) + }) + .collect::>>() + .unwrap(); + + let new_keys = bw_agg + .partition_keys + .iter() + .zip(window_usage) + .filter_map(|(key, (_column, usage))| { + if usage { + Some(update_column_index(key, &schema_mapping)) + } else { + None + } + }) + .collect(); + let plan = Arc::new(BoundedWindowAggExec::try_new( + new_window_exprs, + new_child.plan.clone(), + new_keys, + bw_agg.input_order_mode.clone(), + )?) as _; + let required_columns = collect_columns_in_plan_schema(&plan); + self = ProjectionOptimizer { + plan, + required_columns, + schema_mapping, + children_nodes: vec![new_child], + } + } else { + // Remove the WindowAggExec + self = self.children_nodes.swap_remove(0); + self.required_columns = requirement_map + .into_iter() + .filter_map(|(column, used)| if used { Some(column) } else { None }) + .collect(); + } + } else { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .filter(|col| { + col.index() + < bw_agg.schema().fields().len() - bw_agg.window_expr().len() + }) + .cloned() + .collect(); + } + Ok(self) + } + + /// Compares the required and existing columns in the node, and maps them accordingly. Caller side must + /// ensure that the node extends its own requirements if the node's plan can introduce new requirements. + fn analyze_requirements(&self) -> ColumnRequirements { + let mut requirement_map = HashMap::new(); + let columns_in_schema = collect_columns_in_plan_schema(&self.plan); + columns_in_schema.into_iter().for_each(|col| { + let contains = self.required_columns.contains(&col); + requirement_map.insert(col, contains); + }); + requirement_map + } + + /// Compares the columns required from the left/right child and existing columns in the left/right + /// child. If there is any redundant field, it returns the mapping of columns whether it is required + /// or not. If there is no redundancy, it returns `None` for that child. Caller side must ensure + /// that the join node extends its own requirements if the node's plan can introduce new requirements. + /// Each column refers to its own table schema index, not to the join output schema. + fn analyze_requirements_of_joins( + &self, + left_size: usize, + ) -> (ColumnRequirements, ColumnRequirements) { + let columns_in_schema = + collect_columns_in_plan_schema(&self.children_nodes[0].plan) + .into_iter() + .chain( + collect_columns_in_plan_schema(&self.children_nodes[1].plan) + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)), + ); + let requirement_map = columns_in_schema + .into_iter() + .map(|col| { + if self.required_columns.contains(&col) { + (col, true) + } else { + (col, false) + } + }) + .collect::>(); + + let (requirement_map_left, mut requirement_map_right) = requirement_map + .into_iter() + .partition::, _>(|(col, _)| col.index() < left_size); + + requirement_map_right = requirement_map_right + .into_iter() + .map(|(col, used)| (Column::new(col.name(), col.index() - left_size), used)) + .collect::>(); + + (requirement_map_left, requirement_map_right) + } + + /// If a node is known to have redundant columns, we need to insert a projection to its input. + /// This function takes this node and requirement mapping of this node. Then, defines the projection + /// and constructs the new subtree. The returned objects are the new tree starting from the inserted + /// projection, and the mapping of columns referring to the schemas of pre-insertion and post-insertion. + fn insert_projection( + self, + requirement_map: ColumnRequirements, + ) -> Result<(Self, HashMap)> { + // During the iteration, we construct the ProjectionExec with required columns as the new child, + // and also collect the unused columns to store the index changes after removal of some columns. + let mut unused_columns = HashSet::new(); + let mut projected_exprs = requirement_map + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect::>(); + projected_exprs.sort_by_key(|(expr, _alias)| { + expr.as_any().downcast_ref::().unwrap().index() + }); + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs, + self.plan.children()[0].clone(), + )?) as _; + + let mut new_mapping = HashMap::new(); + for col in self.required_columns.iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter() { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let new_requirements = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + // Required columns must have been extended with self node requirements before this point. + required_columns: new_requirements, + schema_mapping: HashMap::new(), + children_nodes: self.children_nodes, + }; + Ok((inserted_projection, new_mapping)) + } + + /// Multi-child version of `insert_projection` for `UnionExec`'s. + fn insert_multi_projection_below_union( + self, + requirement_map: ColumnRequirements, + ) -> Result<(Vec, HashMap)> { + // During the iteration, we construct the ProjectionExec's with required columns as the new children, + // and also collect the unused columns to store the index changes after removal of some columns. + let mut unused_columns = HashSet::new(); + let mut projected_exprs = requirement_map + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect::>(); + projected_exprs.sort_by_key(|(expr, _alias)| { + expr.as_any().downcast_ref::().unwrap().index() + }); + let inserted_projections = self + .plan + .children() + .into_iter() + .map(|child_plan| { + Ok(Arc::new(ProjectionExec::try_new( + projected_exprs.clone(), + child_plan, + )?) as _) + }) + .collect::>>()?; + + let mut new_mapping = HashMap::new(); + for col in self.required_columns.iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter() { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let new_requirements = inserted_projections + .iter() + .map(|inserted_projection| { + collect_columns_in_plan_schema(inserted_projection) + }) + .collect::>(); + let inserted_projection_nodes = inserted_projections + .into_iter() + .zip(self.children_nodes) + .enumerate() + .map(|(idx, (p, child))| ProjectionOptimizer { + plan: p, + required_columns: new_requirements[idx].clone(), + schema_mapping: HashMap::new(), + children_nodes: vec![child], + }) + .collect(); + Ok((inserted_projection_nodes, new_mapping)) + } + + /// Single child version of `insert_projection` for joins. + fn insert_projection_below_single_child( + self, + requirement_map_left: ColumnRequirements, + children_index: usize, + ) -> Result<(Self, HashMap)> { + let mut unused_columns = HashSet::new(); + // During the iteration, we construct the ProjectionExec with required columns as the new child, + // and also collect the unused columns to store the index changes after removal of some columns. + let mut projected_exprs = requirement_map_left + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect::>(); + projected_exprs.sort_by_key(|(expr, _alias)| { + expr.as_any().downcast_ref::().unwrap().index() + }); + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs.clone(), + self.plan.children()[children_index].clone(), + )?) as _; + + let required_columns = projected_exprs + .iter() + .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) + .collect::>(); + + let mut new_mapping = HashMap::new(); + for col in required_columns.into_iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter() { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let required_columns = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: HashMap::new(), + children_nodes: vec![self.children_nodes[children_index].clone()], + }; + Ok((inserted_projection, new_mapping)) + } + + /// Multi-child version of `insert_projection` for joins. + fn insert_multi_projections_below_join( + self, + left_size: usize, + requirement_map_left: ColumnRequirements, + requirement_map_right: ColumnRequirements, + ) -> Result<(Self, Self, HashMap)> { + let original_right = self.children_nodes[1].plan.clone(); + let (new_left_child, mut left_schema_mapping) = self + .clone() + .insert_projection_below_single_child(requirement_map_left, 0)?; + let (new_right_child, right_schema_mapping) = + self.insert_projection_below_single_child(requirement_map_right, 1)?; + + let new_left_size = new_left_child.plan.schema().fields().len(); + // left_schema_mapping does not need to be change, but it is updated with + // those coming form the right side to represent overall join output mapping. + for (idx, field) in + original_right + .schema() + .fields() + .iter() + .enumerate() + .filter(|(idx, field)| { + let right_projection = new_right_child + .plan + .as_any() + .downcast_ref::() + .unwrap() + .expr() + .iter() + .map(|(expr, _alias)| { + expr.as_any().downcast_ref::().unwrap() + }) + .collect::>(); + right_projection.contains(&&Column::new(field.name(), *idx)) + }) + { + left_schema_mapping.insert( + Column::new(field.name(), idx + left_size), + Column::new(field.name(), idx + new_left_size), + ); + } + for (old, new) in right_schema_mapping.into_iter() { + left_schema_mapping.insert( + Column::new(old.name(), old.index() + left_size), + Column::new(new.name(), new.index() + new_left_size), + ); + } + Ok((new_left_child, new_right_child, left_schema_mapping)) + } + + /// `insert_projection` for windows. + fn insert_projection_below_window( + self, + w_agg: &WindowAggExec, + requirement_map: ColumnRequirements, + ) -> Result<(Self, HashMap, ColumnRequirements)> { + let original_schema_len = w_agg.schema().fields().len(); + let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map + .into_iter() + .partition(|(column, _used)| column.index() < original_schema_len); + let mut unused_columns = HashSet::new(); + + let projected_exprs = base + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect(); + window.iter().for_each(|(col, used)| { + if !used { + unused_columns.insert(col.clone()); + } + }); + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs, + self.plan.children()[0].clone(), + )?) as _; + + let mut new_mapping = HashMap::new(); + for col in self.required_columns.iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter().chain(unused_columns.iter()) { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let new_requirements = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + // Required columns must have been extended with self node requirements before this point. + required_columns: new_requirements, + schema_mapping: HashMap::new(), + children_nodes: self.children_nodes, + }; + Ok((inserted_projection, new_mapping, window)) + } + + /// `insert_projection` for bounded windows. + fn insert_projection_below_bounded_window( + self, + bw_agg: &BoundedWindowAggExec, + requirement_map: ColumnRequirements, + ) -> Result<(Self, HashMap, ColumnRequirements)> { + let original_schema_len = bw_agg.schema().fields().len(); + let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map + .into_iter() + .partition(|(column, _used)| column.index() < original_schema_len); + let mut unused_columns = HashSet::new(); + + let projected_exprs = base + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect(); + window.iter().for_each(|(col, used)| { + if !used { + unused_columns.insert(col.clone()); + } + }); + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs, + self.plan.children()[0].clone(), + )?) as _; + + let mut new_mapping = HashMap::new(); + for col in self.required_columns.iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter().chain(unused_columns.iter()) { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let new_requirements = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + // Required columns must have been extended with self node requirements before this point. + required_columns: new_requirements, + schema_mapping: HashMap::new(), + children_nodes: self.children_nodes, + }; + Ok((inserted_projection, new_mapping, window)) + } + + /// Responsible for updating the node's plan with new children and possibly updated column indices, + /// and for transferring the column mapping to the upper nodes. There is an exception for the + /// projection nodes; they may be removed also in case of being considered as unnecessary, + /// which leads to re-update the mapping after removal. + fn index_updater(mut self: ProjectionOptimizer) -> Result> { + let mut all_mappings = self + .children_nodes + .iter() + .map(|node| node.schema_mapping.clone()) + .collect::>(); + if !all_mappings.iter().all(|map| map.is_empty()) { + // The self plan will update its column indices according to the changes its children schemas. + let plan_copy = self.plan.clone(); + let plan_any = plan_copy.as_any(); + + // These plans do not have any expression related field. + // They simply transfer the mapping to the parent node. + if let Some(_coal_batches) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_coal_parts) = + plan_any.downcast_ref::() + { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_glimit) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_llimit) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_union) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_union) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_cj) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(projection) = plan_any.downcast_ref::() { + self.plan = rewrite_projection( + projection, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + // Rewriting the projection does not change its output schema, + // and projections does not need to transfer the mapping to upper nodes. + } else if let Some(filter) = plan_any.downcast_ref::() { + self.plan = rewrite_filter( + filter.predicate(), + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + self.update_mapping(all_mappings) + } else if let Some(repartition) = plan_any.downcast_ref::() { + self.plan = rewrite_repartition( + repartition.partitioning(), + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + self.update_mapping(all_mappings) + } else if let Some(sort) = plan_any.downcast_ref::() { + self.plan = rewrite_sort( + sort.expr(), + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + self.update_mapping(all_mappings) + } else if let Some(sortp_merge) = + plan_any.downcast_ref::() + { + self.plan = rewrite_sort_preserving_merge( + sortp_merge.expr(), + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + self.update_mapping(all_mappings) + } else if let Some(hj) = plan_any.downcast_ref::() { + let left_size = self.children_nodes[0].plan.schema().fields().len(); + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let new_mapping = left_mapping + .iter() + .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(new.name(), new.index() + left_size), + ) + })) + .collect::>(); + self.plan = rewrite_hash_join( + hj, + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + &new_mapping, + left_size, + )?; + match hj.join_type() { + JoinType::Right + | JoinType::Full + | JoinType::Left + | JoinType::Inner => { + let (new_left, new_right) = + new_mapping.into_iter().partition(|(col_initial, _)| { + col_initial.index() < left_size + }); + all_mappings.push(new_left); + all_mappings.push(new_right); + } + JoinType::LeftSemi | JoinType::LeftAnti => { + all_mappings.push(left_mapping) + } + JoinType::RightAnti | JoinType::RightSemi => { + all_mappings.push(right_mapping) + } + }; + self.update_mapping(all_mappings) + } else if let Some(nlj) = plan_any.downcast_ref::() { + let left_size = self.children_nodes[0].plan.schema().fields().len(); + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let new_mapping = left_mapping + .iter() + .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(new.name(), new.index() + left_size), + ) + })) + .collect::>(); + self.plan = rewrite_nested_loop_join( + nlj, + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + &new_mapping, + left_size, + )?; + all_mappings[0] = match nlj.join_type() { + JoinType::Right + | JoinType::Full + | JoinType::Left + | JoinType::Inner => new_mapping, + JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, + JoinType::RightAnti | JoinType::RightSemi => right_mapping, + }; + self.update_mapping(all_mappings) + } else if let Some(smj) = plan_any.downcast_ref::() { + let left_size = self.children_nodes[0].plan.schema().fields().len(); + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let new_mapping = left_mapping + .iter() + .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(new.name(), new.index() + left_size), + ) + })) + .collect::>(); + self.plan = rewrite_sort_merge_join( + smj, + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + &new_mapping, + left_size, + )?; + all_mappings[0] = match smj.join_type() { + JoinType::Right + | JoinType::Full + | JoinType::Left + | JoinType::Inner => new_mapping, + JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, + JoinType::RightAnti | JoinType::RightSemi => right_mapping, + }; + self.update_mapping(all_mappings) + } else if let Some(shj) = plan_any.downcast_ref::() { + let left_size = self.children_nodes[0].plan.schema().fields().len(); + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let new_mapping = left_mapping + .iter() + .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(new.name(), new.index() + left_size), + ) + })) + .collect::>(); + self.plan = rewrite_symmetric_hash_join( + shj, + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + &new_mapping, + left_size, + )?; + all_mappings[0] = match shj.join_type() { + JoinType::Right + | JoinType::Full + | JoinType::Left + | JoinType::Inner => new_mapping, + JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, + JoinType::RightAnti | JoinType::RightSemi => right_mapping, + }; + self.update_mapping(all_mappings) + } else if let Some(agg) = plan_any.downcast_ref::() { + self.plan = if let Some(updated) = rewrite_aggregate( + agg, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )? { + updated + } else { + return Ok(Transformed::No(self)); + }; + self.update_mapping(all_mappings) + } else if let Some(w_agg) = plan_any.downcast_ref::() { + self.plan = if let Some(updated) = rewrite_window_aggregate( + w_agg, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )? { + updated + } else { + return Ok(Transformed::No(self)); + }; + self.update_mapping(all_mappings) + } else if let Some(bw_agg) = plan_any.downcast_ref::() { + self.plan = if let Some(updated) = rewrite_bounded_window_aggregate( + bw_agg, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )? { + updated + } else { + return Ok(Transformed::No(self)); + }; + self.update_mapping(all_mappings) + } else if let Some(file_sink) = plan_any.downcast_ref::() { + let mapped_exprs = + all_mappings.swap_remove(0).into_iter().collect::>(); + let mut existing_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan) + .into_iter() + .collect_vec(); + existing_columns.sort_by_key(|col| col.index()); + let mut exprs = vec![]; + for idx in 0..existing_columns.len() { + if let Some((initial, _final)) = mapped_exprs + .iter() + .find(|(initial, _final)| initial.index() == idx) + { + exprs.push(( + Arc::new(initial.clone()) as Arc, + initial.name().to_string(), + )); + } else { + exprs.push(( + Arc::new(existing_columns[idx].clone()) + as Arc, + existing_columns[idx].name().to_string(), + )); + } + } + let projection = Arc::new(ProjectionExec::try_new( + exprs, + self.children_nodes[0].plan.clone(), + )?); + let new_child = ProjectionOptimizer { + plan: projection, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(0)], + }; + self.plan = self.plan.with_new_children(vec![new_child.plan.clone()])?; + self.children_nodes = vec![new_child]; + } else { + unreachable!() + } + } else { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + } + + Ok(Transformed::Yes(self)) + } + + fn update_mapping(&mut self, mut child_mappings: Vec>) { + if self.schema_mapping.is_empty() { + self.schema_mapping = child_mappings.swap_remove(0); + } else { + let child_map = child_mappings.swap_remove(0); + self.schema_mapping = self + .schema_mapping + .iter() + .map(|(initial, new)| { + ( + initial.clone(), + child_map.get(&new).cloned().unwrap_or(new.clone()), + ) + }) + .collect() + } + } + + /// After the top-down pass, there may be some unnecessary projections surviving + /// since they assumes themselves as necessary when they are analyzed, but after + /// some optimizations below, they may become unnecessary. This function checks + /// if the projection is still necessary. If it is not so, it is removed, and + /// a new mapping is set to the new node, which is the child of the projection, + /// to transfer the changes resulting from the removal of the projection. + fn try_remove_projection_bottom_up(mut self) -> Result { + let plan = self.plan.clone(); + let Some(projection) = plan.as_any().downcast_ref::() else { + return Ok(self); + }; + // Is the projection really required? First, we need to + // have all column expression in the projection for removal. + if all_alias_free_columns(projection.expr()) { + // Then, check if all columns in the input schema exist after + // the projection. If it is so, we can remove the projection + // since it does not provide any benefit. + let child_columns = collect_columns_in_plan_schema(projection.input()); + let projection_columns = projection + .expr() + .iter() + .map(|(expr, _alias)| { + // We have ensured all expressions are column. + expr.as_any().downcast_ref::().unwrap().clone() + }) + .collect::>(); + if child_columns + .iter() + .all(|child_col| projection_columns.contains(child_col)) + { + // We need to store the existing node's mapping. + let self_mapping = self.schema_mapping; + // Remove the projection node. + self = self.children_nodes.swap_remove(0); + + if self_mapping.is_empty() { + self.schema_mapping = projection + .expr() + .iter() + .enumerate() + .filter_map(|(idx, (col, _alias))| { + let new_column = + col.as_any().downcast_ref::().unwrap(); + if new_column.index() != idx { + Some(( + Column::new(new_column.name(), idx), + new_column.clone(), + )) + } else { + None + } + }) + .collect(); + } else { + self.schema_mapping = self_mapping + .into_iter() + .map(|(expected, updated)| { + ( + expected, + Column::new( + updated.name(), + projection_columns[updated.index()].index(), + ), + ) + }) + .collect() + } + } + } + return Ok(self); + } +} + +impl TreeNode for ProjectionOptimizer { + fn apply_children(&self, op: &mut F) -> Result + where + F: FnMut(&Self) -> Result, + { + for child in &self.children_nodes { + match op(child)? { + VisitRecursion::Continue => {} + VisitRecursion::Skip => return Ok(VisitRecursion::Continue), + VisitRecursion::Stop => return Ok(VisitRecursion::Stop), + } + } + Ok(VisitRecursion::Continue) + } + + fn map_children(mut self, transform: F) -> Result + where + F: FnMut(Self) -> Result, + { + // print_plan(&self.plan); + // println!("self reqs: {:?}", self.required_columns); + // println!("self map: {:?}", self.schema_mapping); + // self.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); + + if self.children_nodes.is_empty() { + Ok(self) + } else { + self.children_nodes = self + .children_nodes + .into_iter() + .map(transform) + .collect::>>()?; + + self = match self.index_updater()? { + Transformed::Yes(updated) => updated, + Transformed::No(not_rewritable) => { + ProjectionOptimizer::new_default(not_rewritable.plan) + } + }; + // After the top-down pass, there may be some unnecessary projections surviving + // since they assumes themselves as necessary when they are analyzed, but after + // some optimizations below, they may become unnecessary. This check is done + // here, and if the projection is regarded as unnecessary, the removal would + // set a new the mapping on the new node, which is the child of the projection. + self = self.try_remove_projection_bottom_up()?; + Ok(self) + } + } +} + +#[derive(Default)] +pub struct OptimizeProjections {} + +impl OptimizeProjections { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} +fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) +} +impl PhysicalOptimizerRule for OptimizeProjections { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + // Collect initial columns requirements from the plan's schema. + let initial_requirements = collect_columns_in_plan_schema(&plan); + let mut optimizer = ProjectionOptimizer::new_default(plan); + // Insert the initial requirements to the root node, and run the rule. + optimizer.required_columns = initial_requirements.clone(); + let mut optimized = optimizer.transform_down(&|o| { + o.adjust_node_with_requirements().map(Transformed::Yes) + })?; + // Ensure the final optimized plan satisfies the initial schema requirements. + optimized = satisfy_initial_schema(optimized, initial_requirements)?; + + // TODO: Remove this check to tests + crosscheck_helper(optimized.clone())?; + + Ok(optimized.plan) + } + + fn name(&self) -> &str { + "OptimizeProjections" + } + + fn schema_check(&self) -> bool { + true + } +} + +// TODO: Remove this to tests +pub fn crosscheck_helper(context: ProjectionOptimizer) -> Result<()> { + context.transform_up(&|node| { + assert_eq!(node.children_nodes.len(), node.plan.children().len()); + if !node.children_nodes.is_empty() { + assert_eq!( + get_plan_string(&node.plan), + get_plan_string(&node.plan.clone().with_new_children( + node.children_nodes.iter().map(|c| c.plan.clone()).collect() + )?) + ); + } + Ok(Transformed::No(node)) + })?; + + Ok(()) +} + +/// Ensures that the output schema `po` matches the `initial_requirements`. +/// If the `schema_mapping` of `po` indicates that some columns have been re-mapped, +/// a new projection is added to restore the initial column order and indices. +fn satisfy_initial_schema( + po: ProjectionOptimizer, + initial_requirements: HashSet, +) -> Result { + if collect_columns_in_plan_schema(&po.plan) == initial_requirements { + // The initial schema is already satisfied, no further action required. + Ok(po) + } else { + // Collect expressions for the final projection to match the initial requirements. + let mut initial_requirements_vec = + initial_requirements.clone().into_iter().collect_vec(); + initial_requirements_vec.sort_by_key(|expr| expr.index()); + let projected_exprs = initial_requirements_vec + .iter() + .map(|col| { + // If there is a change, get the new index. + let column_index = po.schema_mapping.get(&col).unwrap_or(&col).index(); + let new_col = Arc::new(Column::new(col.name(), column_index)) + as Arc; + (new_col, col.name().to_string()) + }) + .collect::>(); + + // Create the final projection to align with the initial schema. + let final_projection = + Arc::new(ProjectionExec::try_new(projected_exprs, po.plan.clone())?); + + // Return a new ProjectionOptimizer with the final projection, resetting the schema mapping. + Ok(ProjectionOptimizer { + plan: final_projection, + required_columns: initial_requirements, + schema_mapping: HashMap::new(), // Reset schema mapping as we've now satisfied the initial schema + children_nodes: vec![po], // Keep the original node as the child + }) + } +} + +/// Iterates over all columns and returns true if all columns are required. +fn all_columns_required(requirement_map: &ColumnRequirements) -> bool { + requirement_map.iter().all(|(_k, v)| *v) +} + +fn window_agg_required( + original_schema_len: usize, + requirements: &ColumnRequirements, +) -> bool { + requirements + .iter() + .filter(|(column, _used)| column.index() >= original_schema_len) + .any(|(_column, used)| *used) +} + +// If an expression is not trivial and it is referred more than 1, +// unification will not be beneficial as going against caching mechanism +// for non-trivial computations. See the discussion: +// https://github.com/apache/arrow-datafusion/issues/8296 +fn caching_projections( + projection: &ProjectionExec, + child_projection: &ProjectionExec, +) -> bool { + let mut column_ref_map: HashMap = HashMap::new(); + // Collect the column references' usage in the parent projection. + projection.expr().iter().for_each(|(expr, _)| { + expr.apply(&mut |expr| { + Ok({ + if let Some(column) = expr.as_any().downcast_ref::() { + *column_ref_map.entry(column.clone()).or_default() += 1; + } + VisitRecursion::Continue + }) + }) + .unwrap(); + }); + column_ref_map.iter().any(|(column, count)| { + *count > 1 && !is_expr_trivial(&child_projection.expr()[column.index()].0) + }) +} + +/// Checks if the given expression is trivial. +/// An expression is considered trivial if it is either a `Column` or a `Literal`. +fn is_expr_trivial(expr: &Arc) -> bool { + expr.as_any().downcast_ref::().is_some() + || expr.as_any().downcast_ref::().is_some() +} + +/// Given the expression set of a projection, checks if the projection causes +/// any renaming or constructs a non-`Column` physical expression. +fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { + exprs.iter().all(|(expr, alias)| { + expr.as_any() + .downcast_ref::() + .map(|column| column.name() == alias) + .unwrap_or(false) + }) +} + +/// Updates a source provider's projected columns according to the given +/// projection operator's expressions. To use this function safely, one must +/// ensure that all expressions are `Column` expressions without aliases. +fn new_projections_for_columns( + projection: &[&Column], + source: &Option>, +) -> Vec { + projection + .iter() + .filter_map(|col| source.as_ref().map(|proj| proj[col.index()])) + .collect() +} + +#[derive(Debug, PartialEq)] +enum RewriteState { + /// The expression is unchanged. + Unchanged, + /// Some part of the expression has been rewritten + RewrittenValid, + /// Some part of the expression has been rewritten, but some column + /// references could not be. + RewrittenInvalid, +} + +/// The function operates in two modes: +/// +/// 1) When `sync_with_child` is `true`: +/// +/// The function updates the indices of `expr` if the expression resides +/// in the input plan. For instance, given the expressions `a@1 + b@2` +/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are +/// updated to `a@0 + b@1` and `c@2`. +/// +/// 2) When `sync_with_child` is `false`: +/// +/// The function determines how the expression would be updated if a projection +/// was placed before the plan associated with the expression. If the expression +/// cannot be rewritten after the projection, it returns `None`. For example, +/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with +/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes +/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. +fn update_expr( + expr: &Arc, + projected_exprs: &[(Arc, String)], + sync_with_child: bool, +) -> Result>> { + let mut state = RewriteState::Unchanged; + let new_expr = expr + .clone() + .transform_up_mut(&mut |expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + if sync_with_child { + state = RewriteState::RewrittenValid; + // Update the index of `column`: + Ok(Transformed::Yes(projected_exprs[column.index()].0.clone())) + } else { + // default to invalid, in case we can't find the relevant column + state = RewriteState::RewrittenInvalid; + // Determine how to update `column` to accommodate `projected_exprs` + projected_exprs + .iter() + .enumerate() + .find_map(|(index, (projected_expr, alias))| { + projected_expr.as_any().downcast_ref::().and_then( + |projected_column| { + column.name().eq(projected_column.name()).then(|| { + state = RewriteState::RewrittenValid; + Arc::new(Column::new(alias, index)) as _ + }) + }, + ) + }) + .map_or_else( + || Ok(Transformed::No(expr)), + |c| Ok(Transformed::Yes(c)), + ) + } + }); + new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) +} + +/// Given mapping representing the initial and new index values, +/// it updates the indices of columns in the [`PhysicalExpr`]. +fn update_column_index( + expr: &Arc, + mapping: &HashMap, +) -> Arc { + let mut state = RewriteState::Unchanged; + let new_expr = expr + .clone() + .transform_up_mut(&mut |expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + state = RewriteState::RewrittenValid; + // Update the index of `column`: + if let Some(updated) = mapping.get(column) { + Ok(Transformed::Yes(Arc::new(updated.clone()) as _)) + } else { + Ok(Transformed::No(expr.clone())) + } + }) + .unwrap(); + new_expr +} + +/// Collects all fields of the schema for a given plan in [`Column`] form. +fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet { + plan.schema() + .fields() + .iter() + .enumerate() + .map(|(i, f)| Column::new(f.name(), i)) + .collect() +} + +/// Collects all columns in the join's equivalence and non-equivalence conditions as they are seen at the join output. +/// This means that columns from left table appear as they are, and right table column indices increased by left table size. +fn collect_columns_in_join_conditions( + on: &[(Arc, Arc)], + filter: Option<&JoinFilter>, + left_size: usize, + join_left_schema: SchemaRef, + join_right_schema: SchemaRef, +) -> HashSet { + let equivalence_columns = on + .iter() + .flat_map(|(col_left, col_right)| { + let left_columns = collect_columns(col_left); + let right_columns = collect_columns(col_right); + let mut state = RewriteState::Unchanged; + let right_columns = right_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect_vec(); + left_columns.into_iter().chain(right_columns).collect_vec() + }) + .collect::>(); + let non_equivalence_columns = filter + .map(|filter| { + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => Column::new( + join_left_schema.fields()[col_idx.index].name(), + col_idx.index, + ), + JoinSide::Right => Column::new( + join_right_schema.fields()[col_idx.index].name(), + col_idx.index + left_size, + ), + }) + .collect::>() + }) + .unwrap_or_default(); + equivalence_columns + .into_iter() + .chain(non_equivalence_columns.into_iter()) + .collect() +} + +/// Updates the equivalence conditions of the joins according to the new indices of columns. +fn update_equivalence_conditions( + on: &[(Arc, Arc)], + requirement_map_left: &ColumnRequirements, + requirement_map_right: &ColumnRequirements, +) -> JoinOn { + on.iter() + .map(|(left_col, right_col)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left_col + .clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new(Column::new( + column.name(), + column.index() + - removed_column_count( + requirement_map_left, + column.index(), + ), + )))) + }) + .unwrap(), + right_col + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new(Column::new( + column.name(), + column.index() + - removed_column_count( + requirement_map_right, + column.index(), + ), + )))) + }) + .unwrap(), + ) + }) + .collect() +} + +/// Updates the [`JoinFilter`] according to the new indices of columns. +fn update_non_equivalence_conditions( + filter: Option<&JoinFilter>, + requirement_map_left: &ColumnRequirements, + requirement_map_right: &ColumnRequirements, +) -> Option { + filter.map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: col_idx.index + - removed_column_count(requirement_map_left, col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: col_idx.index + - removed_column_count(requirement_map_right, col_idx.index), + side: JoinSide::Right, + }, + }) + .collect(), + filter.schema().clone(), + ) + }) +} + +/// Calculates how many index of the given column decreases becasue of +/// the removed columns which reside on the left side of that given column. +fn removed_column_count( + requirement_map: &ColumnRequirements, + column_index: usize, +) -> usize { + let mut left_skipped_columns = 0; + for unused_col in + requirement_map.iter().filter_map( + |(col, used)| { + if *used { + None + } else { + Some(col) + } + }, + ) + { + if unused_col.index() < column_index { + left_skipped_columns += 1; + } + } + left_skipped_columns +} + +fn rewrite_projection( + projection: &ProjectionExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + ProjectionExec::try_new( + projection + .expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.clone())) + .collect::>(), + input_plan, + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_filter( + predicate: &Arc, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + FilterExec::try_new(update_column_index(predicate, mapping), input_plan) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_repartition( + partitioning: &Partitioning, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { + let new_exprs = exprs + .iter() + .map(|expr| update_column_index(expr, &mapping)) + .collect::>(); + Partitioning::Hash(new_exprs, *size) + } else { + partitioning.clone() + }; + RepartitionExec::try_new(input_plan, new_partitioning).map(|plan| Arc::new(plan) as _) +} + +fn rewrite_sort( + sort_expr: &[PhysicalSortExpr], + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + let new_sort_exprs = sort_expr + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, &mapping), + options: sort_expr.options, + }) + .collect::>(); + Ok(Arc::new(SortExec::new(new_sort_exprs, input_plan)) as _) +} + +fn rewrite_sort_preserving_merge( + sort_expr: &[PhysicalSortExpr], + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + let new_sort_exprs = sort_expr + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, &mapping), + options: sort_expr.options, + }) + .collect::>(); + Ok(Arc::new(SortExec::new(new_sort_exprs, input_plan)) as _) +} + +fn rewrite_hash_join( + hj: &HashJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + mapping: &HashMap, + left_size: usize, +) -> Result> { + let new_on = hj + .on() + .into_iter() + .map(|(left, right)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left.clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index( + &(Arc::new(column.clone()) as _), + &mapping, + ) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + right + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index( + &(Arc::new(column.clone()) as _), + &mapping, + ) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + ) + }) + .collect(); + let new_filter = hj.filter().map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + left_size + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + HashJoinExec::try_new( + left_input_plan, + right_input_plan, + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_nested_loop_join( + nlj: &NestedLoopJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + mapping: &HashMap, + left_size: usize, +) -> Result> { + let new_filter = nlj.filter().map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + left_size + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + NestedLoopJoinExec::try_new( + left_input_plan, + right_input_plan, + new_filter, + nlj.join_type(), + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_sort_merge_join( + smj: &SortMergeJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + mapping: &HashMap, + left_size: usize, +) -> Result> { + let new_on = smj + .on() + .into_iter() + .map(|(left, right)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left.clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index( + &(Arc::new(column.clone()) as _), + &mapping, + ) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + right + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index( + &(Arc::new(column.clone()) as _), + &mapping, + ) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + ) + }) + .collect(); + let new_filter = smj.filter.as_ref().map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + left_size + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + SortMergeJoinExec::try_new( + left_input_plan, + right_input_plan, + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_symmetric_hash_join( + shj: &SymmetricHashJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + mapping: &HashMap, + left_size: usize, +) -> Result> { + let new_on = shj + .on() + .into_iter() + .map(|(left, right)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left.clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index(&(left.clone()), &mapping) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + right + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index(&(right.clone()), &mapping) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + ) + }) + .collect(); + let new_filter = shj.filter().map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + left_size + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + SymmetricHashJoinExec::try_new( + left_input_plan, + right_input_plan, + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + // TODO: update these + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_aggregate( + agg: &AggregateExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result>> { + let new_group_by = PhysicalGroupBy::new( + agg.group_expr() + .expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) + .collect(), + agg.group_expr() + .null_expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) + .collect(), + agg.group_expr().groups().to_vec(), + ); + let new_agg_expr = if let Some(new_agg_expr) = agg + .aggr_expr() + .iter() + .map(|aggr_expr| { + aggr_expr.clone().with_new_expressions( + aggr_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(), + ) + }) + .collect::>>() + { + new_agg_expr + } else { + return Ok(None); + }; + let new_filter = agg + .filter_expr() + .iter() + .map(|opt_expr| { + opt_expr + .clone() + .map(|expr| update_column_index(&expr, mapping)) + }) + .collect(); + AggregateExec::try_new( + *agg.mode(), + new_group_by, + new_agg_expr, + new_filter, + input_plan, + agg.input_schema(), + ) + .map(|plan| Some(Arc::new(plan) as _)) +} + +fn rewrite_window_aggregate( + w_agg: &WindowAggExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result>> { + let new_window = if let Some(new_window) = w_agg + .window_expr() + .iter() + .map(|window_expr| { + window_expr.clone().with_new_expressions( + window_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(), + ) + }) + .collect::>>() + { + new_window + } else { + return Ok(None); + }; + let new_partition_keys = w_agg + .partition_keys + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(); + WindowAggExec::try_new(new_window, input_plan, new_partition_keys) + .map(|plan| Some(Arc::new(plan) as _)) +} + +fn rewrite_bounded_window_aggregate( + bw_agg: &BoundedWindowAggExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result>> { + let new_window = if let Some(new_window) = bw_agg + .window_expr() + .iter() + .map(|window_expr| { + window_expr.clone().with_new_expressions( + window_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(), + ) + }) + .collect::>>() + { + new_window + } else { + return Ok(None); + }; + let new_partition_keys = bw_agg + .partition_keys + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(); + BoundedWindowAggExec::try_new( + new_window, + input_plan, + new_partition_keys, + bw_agg.input_order_mode.clone(), + ) + .map(|plan| Some(Arc::new(plan) as _)) +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use crate::datasource::file_format::file_compression_type::FileCompressionType; + use crate::datasource::listing::PartitionedFile; + use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; + use crate::execution::context::SessionContext; + use crate::physical_optimizer::optimize_projections::{ + update_expr, OptimizeProjections, + }; + use crate::physical_optimizer::PhysicalOptimizerRule; + use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; + use crate::physical_plan::filter::FilterExec; + use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; + use crate::physical_plan::joins::StreamJoinPartitionMode; + use crate::physical_plan::projection::ProjectionExec; + use crate::physical_plan::repartition::RepartitionExec; + use crate::physical_plan::sorts::sort::SortExec; + use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; + use crate::physical_plan::ExecutionPlan; + + use arrow::util::pretty::print_batches; + use arrow_schema::{DataType, Field, Schema, SortOptions}; + use datafusion_common::config::ConfigOptions; + use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; + use datafusion_execution::config::SessionConfig; + use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_expr::{ColumnarValue, Operator}; + use datafusion_physical_expr::expressions::{ + BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, + }; + use datafusion_physical_expr::{ + Partitioning, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, + }; + use datafusion_physical_plan::get_plan_string; + use datafusion_physical_plan::joins::SymmetricHashJoinExec; + use datafusion_physical_plan::union::UnionExec; + + use super::print_plan; + + fn create_simple_csv_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])); + Arc::new(CsvExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::new_unknown(&schema), + projection: Some(vec![0, 1, 2, 3, 4]), + limit: None, + table_partition_cols: vec![], + output_ordering: vec![vec![]], + }, + false, + 0, + 0, + None, + FileCompressionType::UNCOMPRESSED, + )) + } + + fn create_projecting_csv_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])); + Arc::new(CsvExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::new_unknown(&schema), + projection: Some(vec![3, 0, 1]), + limit: None, + table_partition_cols: vec![], + output_ordering: vec![vec![]], + }, + false, + 0, + 0, + None, + FileCompressionType::UNCOMPRESSED, + )) + } + + #[test] + fn test_update_matching_exprs() -> Result<()> { + let exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Divide, + Arc::new(Column::new("e", 5)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 3)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 0)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + None, + false, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 2))), + vec![ + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 2)), + Operator::Plus, + Arc::new(Column::new("e", 5)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 5)), + Operator::Plus, + Arc::new(Column::new("d", 2)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Modulo, + Arc::new(Column::new("e", 5)), + ))), + )?), + ]; + let child: Vec<(Arc, String)> = vec![ + (Arc::new(Column::new("c", 2)), "c".to_owned()), + (Arc::new(Column::new("b", 1)), "b".to_owned()), + (Arc::new(Column::new("d", 3)), "d".to_owned()), + (Arc::new(Column::new("a", 0)), "a".to_owned()), + (Arc::new(Column::new("f", 5)), "f".to_owned()), + (Arc::new(Column::new("e", 4)), "e".to_owned()), + ]; + let expected_exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Divide, + Arc::new(Column::new("e", 4)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 0)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 2)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + None, + false, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 3))), + vec![ + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 3)), + Operator::Plus, + Arc::new(Column::new("e", 4)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("d", 3)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Modulo, + Arc::new(Column::new("e", 4)), + ))), + )?), + ]; + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + assert!(update_expr(&expr, &child, true)? + .unwrap() + .eq(&expected_expr)); + } + Ok(()) + } + + #[test] + fn test_update_projected_exprs() -> Result<()> { + let exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Divide, + Arc::new(Column::new("e", 5)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 3)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 0)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + None, + false, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 2))), + vec![ + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 2)), + Operator::Plus, + Arc::new(Column::new("e", 5)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 5)), + Operator::Plus, + Arc::new(Column::new("d", 2)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Modulo, + Arc::new(Column::new("e", 5)), + ))), + )?), + ]; + let projected_exprs: Vec<(Arc, String)> = vec![ + (Arc::new(Column::new("a", 0)), "a".to_owned()), + (Arc::new(Column::new("b", 1)), "b_new".to_owned()), + (Arc::new(Column::new("c", 2)), "c".to_owned()), + (Arc::new(Column::new("d", 3)), "d_new".to_owned()), + (Arc::new(Column::new("e", 4)), "e".to_owned()), + (Arc::new(Column::new("f", 5)), "f_new".to_owned()), + ]; + let expected_exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Divide, + Arc::new(Column::new("e", 4)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 0)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_new", 1)), + Operator::Divide, + Arc::new(Column::new("c", 2)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Divide, + Arc::new(Column::new("b_new", 1)), + )), + ], + DataType::Int32, + None, + false, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d_new", 3))), + vec![ + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d_new", 3)), + Operator::Plus, + Arc::new(Column::new("e", 4)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("d_new", 3)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Modulo, + Arc::new(Column::new("e", 4)), + ))), + )?), + ]; + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + assert!(update_expr(&expr, &projected_exprs, false)? + .unwrap() + .eq(&expected_expr)); + } + Ok(()) + } + + #[test] + fn test_csv_after_projection() -> Result<()> { + let csv = create_projecting_csv_exec(); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 2)), "b".to_string()), + (Arc::new(Column::new("d", 0)), "d".to_string()), + ], + csv.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@2 as b, d@0 as d]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[d, a, b], has_header=false", + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "CsvExec: file_groups={1 group: [[x]]}, projection=[b, d], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_projection_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let child_projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("e", 4)), "new_e".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("b", 1)), "new_b".to_string()), + ], + csv.clone(), + )?); + let top_projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("new_b", 3)), "new_b".to_string()), + ( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_e", 1)), + )), + "binary".to_string(), + ), + (Arc::new(Column::new("new_b", 3)), "newest_b".to_string()), + ], + child_projection.clone(), + )?); + let initial = get_plan_string(&top_projection); + let expected_initial = [ + "ProjectionExec: expr=[new_b@3 as new_b, c@0 + new_e@1 as binary, new_b@3 as newest_b]", + " ProjectionExec: expr=[c@2 as c, e@4 as new_e, a@0 as a, b@1 as new_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(top_projection, &ConfigOptions::new())?; + let expected = [ + "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_coalesce_partitions_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let coalesce_partitions: Arc = + Arc::new(CoalescePartitionsExec::new(csv)); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("a", 0)), "a_new".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ], + coalesce_partitions, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", + " CoalescePartitionsExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@2 as d]", + " CoalescePartitionsExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, d], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_filter_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let predicate = Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Minus, + Arc::new(Column::new("a", 0)), + )), + Operator::Gt, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 3)), + Operator::Minus, + Arc::new(Column::new("a", 0)), + )), + )); + let filter: Arc = + Arc::new(FilterExec::try_new(predicate, csv)?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("a", 0)), "a_new".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ], + filter.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", + " FilterExec: b@1 - a@0 > d@3 - a@0", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@2 as d]", + " FilterExec: b@1 - a@0 > d@2 - a@0", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, d], has_header=false"]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_join_after_projection() -> Result<()> { + let left_csv = create_simple_csv_exec(); + let right_csv = create_simple_csv_exec(); + let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( + left_csv, + right_csv, + vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], + // b_left-(1+a_right)<=a_right+c_left + Some(JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_left_inter", 0)), + Operator::Minus, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + Operator::Plus, + Arc::new(Column::new("a_right_inter", 1)), + )), + )), + Operator::LtEq, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a_right_inter", 1)), + Operator::Plus, + Arc::new(Column::new("c_left_inter", 2)), + )), + )), + vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ], + Schema::new(vec![ + Field::new("b_left_inter", DataType::Int32, true), + Field::new("a_right_inter", DataType::Int32, true), + Field::new("c_left_inter", DataType::Int32, true), + ]), + )), + &JoinType::Inner, + true, + None, + None, + StreamJoinPartitionMode::SinglePartition, + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), + (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), + (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), + (Arc::new(Column::new("a", 5)), "a_from_right".to_string()), + (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), + ], + join, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", + " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@3 as a_from_right, c@4 as c_from_right]", + " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, c], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + let expected_filter_col_ind = vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ]; + assert_eq!( + expected_filter_col_ind, + after_optimize.children()[0] + .as_any() + .downcast_ref::() + .unwrap() + .filter() + .unwrap() + .column_indices() + ); + Ok(()) + } + + #[test] + fn test_repartition_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let repartition: Arc = Arc::new(RepartitionExec::try_new( + csv, + Partitioning::Hash( + vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("d", 3)), + ], + 6, + ), + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 1)), "b_new".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("d", 3)), "d_new".to_string()), + ], + repartition, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", + " RepartitionExec: partitioning=Hash([a@0, b@1, d@3], 6), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@2 as d_new]", + " RepartitionExec: partitioning=Hash([a@0, b@1, d@2], 6), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, d], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_sort_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(SortExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: SortOptions::default(), + }, + ], + csv.clone(), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_sort_preserving_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: SortOptions::default(), + }, + ], + csv.clone(), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_union_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let union: Arc = + Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + union.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " UnionExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " UnionExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_optimize_projections_filter_sort() -> Result<()> { + /* + INITIAL PLAN: + FilterExec(sum > 0): |sum@0 | + ProjectionExec: |c@2+x@0 as sum | + ProjectionExec: |x@2 |x@0 |c@1 | + SortExec(c@1, x@2): |x@0 |c@1 |x@2 | + ProjectionExec: |x@1 |c@0 |a@2 as x | + ProjectionExec: |c@2 |e@4 as x |a@0 | + CsvExec: |a |b |c |d |e | + ============================================================================================================= + OPTIMIZED PLAN: + FilterExec(sum > 0): |sum@0 | + ProjectionExec: |c@0+x@1 as sum | + SortExec(c@0, x@1): |c@0 |x@1 | + ProjectionExec: |c@2 |a@0 as x | + CsvExec: |a |b |c |d |e | + */ + let csv = create_simple_csv_exec(); + let projection1 = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("e", 4)), "x".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + ], + csv, + )?); + let projection2 = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("x", 1)), "x".to_string()), + (Arc::new(Column::new("c", 0)), "c".to_string()), + (Arc::new(Column::new("a", 2)), "x".to_string()), + ], + projection1, + )?); + let sort = Arc::new(SortExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("c", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("x", 2)), + options: SortOptions::default(), + }, + ], + projection2, + )); + let projection3 = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("x", 2)), "x".to_string()), + (Arc::new(Column::new("x", 0)), "x".to_string()), + (Arc::new(Column::new("c", 1)), "c".to_string()), + ], + sort, + )?); + let projection4 = Arc::new(ProjectionExec::try_new( + vec![( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("x", 0)), + )), + "sum".to_string(), + )], + projection3, + )?); + let filter = Arc::new(FilterExec::try_new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("sum", 0)), + Operator::Gt, + Arc::new(Literal::new(ScalarValue::Int32(Some(0)))), + )), + projection4, + )?) as Arc; + let initial = get_plan_string(&filter); + let expected_initial = [ + "FilterExec: sum@0 > 0", + " ProjectionExec: expr=[c@2 + x@0 as sum]", + " ProjectionExec: expr=[x@2 as x, x@0 as x, c@1 as c]", + " SortExec: expr=[c@1 ASC,x@2 ASC]", + " ProjectionExec: expr=[x@1 as x, c@0 as c, a@2 as x]", + " ProjectionExec: expr=[c@2 as c, e@4 as x, a@0 as a]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(filter, &ConfigOptions::new())?; + let expected = [ + "FilterExec: sum@0 > 0", + " ProjectionExec: expr=[c@0 + x@1 as sum]", + " SortExec: expr=[c@0 ASC,x@1 ASC]", + " ProjectionExec: expr=[c@2 as c, a@0 as x]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[tokio::test] + async fn test_trivial() -> Result<()> { + let mut config = SessionConfig::new() + .with_target_partitions(2) + .with_batch_size(4096); + let ctx = SessionContext::with_config(config); + let _dataframe = ctx + .sql( + "CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL + ) + STORED AS CSV + WITH HEADER ROW + LOCATION '/Users/berkaysahin/Desktop/datafusion-upstream/testing/data/csv/aggregate_test_100.csv'", + ) + .await?; + + let dataframe = ctx + .sql( + "SELECT c2, COUNT(*) +FROM (SELECT c2 +FROM aggregate_test_100 +ORDER BY c1, c2) +GROUP BY c2;", + ) + .await?; + let physical_plan = dataframe.clone().create_physical_plan().await?; + let batches = dataframe.collect().await?; + let _ = print_plan(&physical_plan); + let _ = print_batches(&batches); + Ok(()) + } +} diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index f8c82576e254..ab7fe59e36e8 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -19,7 +19,6 @@ use std::sync::Arc; -use super::projection_pushdown::ProjectionPushdown; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; @@ -28,6 +27,7 @@ use crate::physical_optimizer::enforce_distribution::EnforceDistribution; 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::optimize_projections::OptimizeProjections; use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_optimizer::pipeline_checker::PipelineChecker; use crate::physical_optimizer::topk_aggregation::TopKAggregation; @@ -113,13 +113,10 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), - // The ProjectionPushdown rule tries to push projections towards - // the sources in the execution plan. As a result of this process, - // a projection can disappear if it reaches the source providers, and - // sequential projections can merge into one. Even if these two cases - // are not present, the load of executors such as join or union will be - // reduced by narrowing their input tables. - Arc::new(ProjectionPushdown::new()), + // OptimizeProjections rule aims achieving the most effective use of projections + // in plans. It ensures that query plans are free from unnecessary projections + // and that no unused columns are propagated unnecessarily between plans. + Arc::new(OptimizeProjections::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs deleted file mode 100644 index 79d22374f9c2..000000000000 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ /dev/null @@ -1,2446 +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. - -//! This file implements the `ProjectionPushdown` physical optimization rule. -//! The function [`remove_unnecessary_projections`] tries to push down all -//! projections one by one if the operator below is amenable to this. If a -//! projection reaches a source, it can even dissappear from the plan entirely. - -use std::collections::HashMap; -use std::sync::Arc; - -use super::output_requirements::OutputRequirementExec; -use super::PhysicalOptimizerRule; -use crate::datasource::physical_plan::CsvExec; -use crate::error::Result; -use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; -use crate::physical_plan::joins::{ - CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, - SymmetricHashJoinExec, -}; -use crate::physical_plan::memory::MemoryExec; -use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort::SortExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::{Distribution, ExecutionPlan}; - -use arrow_schema::SchemaRef; -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{DataFusionError, JoinSide}; -use datafusion_physical_expr::expressions::{Column, Literal}; -use datafusion_physical_expr::{ - Partitioning, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, - PhysicalSortRequirement, -}; -use datafusion_physical_plan::streaming::StreamingTableExec; -use datafusion_physical_plan::union::UnionExec; - -use itertools::Itertools; - -/// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to -/// remove or swap with its child. -#[derive(Default)] -pub struct ProjectionPushdown {} - -impl ProjectionPushdown { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl PhysicalOptimizerRule for ProjectionPushdown { - fn optimize( - &self, - plan: Arc, - _config: &ConfigOptions, - ) -> Result> { - plan.transform_down(&remove_unnecessary_projections) - } - - fn name(&self) -> &str { - "ProjectionPushdown" - } - - fn schema_check(&self) -> bool { - true - } -} - -/// This function checks if `plan` is a [`ProjectionExec`], and inspects its -/// input(s) to test whether it can push `plan` under its input(s). This function -/// will operate on the entire tree and may ultimately remove `plan` entirely -/// by leveraging source providers with built-in projection capabilities. -pub fn remove_unnecessary_projections( - plan: Arc, -) -> Result>> { - let maybe_modified = if let Some(projection) = - plan.as_any().downcast_ref::() - { - // If the projection does not cause any change on the input, we can - // safely remove it: - if is_projection_removable(projection) { - return Ok(Transformed::Yes(projection.input().clone())); - } - // If it does, check if we can push it under its child(ren): - let input = projection.input().as_any(); - if let Some(csv) = input.downcast_ref::() { - try_swapping_with_csv(projection, csv) - } else if let Some(memory) = input.downcast_ref::() { - try_swapping_with_memory(projection, memory)? - } else if let Some(child_projection) = input.downcast_ref::() { - let maybe_unified = try_unifying_projections(projection, child_projection)?; - return if let Some(new_plan) = maybe_unified { - // To unify 3 or more sequential projections: - remove_unnecessary_projections(new_plan) - } else { - Ok(Transformed::No(plan)) - }; - } else if let Some(output_req) = input.downcast_ref::() { - try_swapping_with_output_req(projection, output_req)? - } else if input.is::() { - try_swapping_with_coalesce_partitions(projection)? - } else if let Some(filter) = input.downcast_ref::() { - try_swapping_with_filter(projection, filter)? - } else if let Some(repartition) = input.downcast_ref::() { - try_swapping_with_repartition(projection, repartition)? - } else if let Some(sort) = input.downcast_ref::() { - try_swapping_with_sort(projection, sort)? - } else if let Some(spm) = input.downcast_ref::() { - try_swapping_with_sort_preserving_merge(projection, spm)? - } else if let Some(union) = input.downcast_ref::() { - try_pushdown_through_union(projection, union)? - } else if let Some(hash_join) = input.downcast_ref::() { - try_pushdown_through_hash_join(projection, hash_join)? - } else if let Some(cross_join) = input.downcast_ref::() { - try_swapping_with_cross_join(projection, cross_join)? - } else if let Some(nl_join) = input.downcast_ref::() { - try_swapping_with_nested_loop_join(projection, nl_join)? - } else if let Some(sm_join) = input.downcast_ref::() { - try_swapping_with_sort_merge_join(projection, sm_join)? - } else if let Some(sym_join) = input.downcast_ref::() { - try_swapping_with_sym_hash_join(projection, sym_join)? - } else if let Some(ste) = input.downcast_ref::() { - try_swapping_with_streaming_table(projection, ste)? - } else { - // If the input plan of the projection is not one of the above, we - // conservatively assume that pushing the projection down may hurt. - // When adding new operators, consider adding them here if you - // think pushing projections under them is beneficial. - None - } - } else { - return Ok(Transformed::No(plan)); - }; - - Ok(maybe_modified.map_or(Transformed::No(plan), Transformed::Yes)) -} - -/// Tries to embed `projection` to its input (`csv`). If possible, returns -/// [`CsvExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_csv( - projection: &ProjectionExec, - csv: &CsvExec, -) -> Option> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into CsvExec, but it would be an overlap of their responsibility. - all_alias_free_columns(projection.expr()).then(|| { - let mut file_scan = csv.base_config().clone(); - let new_projections = new_projections_for_columns( - projection, - &file_scan - .projection - .unwrap_or((0..csv.schema().fields().len()).collect()), - ); - file_scan.projection = Some(new_projections); - - Arc::new(CsvExec::new( - file_scan, - csv.has_header(), - csv.delimiter(), - csv.quote(), - csv.escape(), - csv.file_compression_type, - )) as _ - }) -} - -/// Tries to embed `projection` to its input (`memory`). If possible, returns -/// [`MemoryExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_memory( - projection: &ProjectionExec, - memory: &MemoryExec, -) -> Result>> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into MemoryExec, but it would be an overlap of their responsibility. - all_alias_free_columns(projection.expr()) - .then(|| { - let all_projections = (0..memory.schema().fields().len()).collect(); - let new_projections = new_projections_for_columns( - projection, - memory.projection().as_ref().unwrap_or(&all_projections), - ); - - MemoryExec::try_new( - memory.partitions(), - memory.original_schema(), - Some(new_projections), - ) - .map(|e| Arc::new(e) as _) - }) - .transpose() -} - -/// Tries to embed `projection` to its input (`streaming table`). -/// If possible, returns [`StreamingTableExec`] as the top plan. Otherwise, -/// returns `None`. -fn try_swapping_with_streaming_table( - projection: &ProjectionExec, - streaming_table: &StreamingTableExec, -) -> Result>> { - if !all_alias_free_columns(projection.expr()) { - return Ok(None); - } - - let streaming_table_projections = streaming_table - .projection() - .as_ref() - .map(|i| i.as_ref().to_vec()); - let new_projections = new_projections_for_columns( - projection, - &streaming_table_projections - .unwrap_or((0..streaming_table.schema().fields().len()).collect()), - ); - - let mut lex_orderings = vec![]; - for lex_ordering in streaming_table.projected_output_ordering().into_iter() { - let mut orderings = vec![]; - for order in lex_ordering { - let Some(new_ordering) = update_expr(&order.expr, projection.expr(), false)? - else { - return Ok(None); - }; - orderings.push(PhysicalSortExpr { - expr: new_ordering, - options: order.options, - }); - } - lex_orderings.push(orderings); - } - - StreamingTableExec::try_new( - streaming_table.partition_schema().clone(), - streaming_table.partitions().clone(), - Some(new_projections.as_ref()), - lex_orderings, - streaming_table.is_infinite(), - ) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Unifies `projection` with its input (which is also a [`ProjectionExec`]). -fn try_unifying_projections( - projection: &ProjectionExec, - child: &ProjectionExec, -) -> Result>> { - let mut projected_exprs = vec![]; - let mut column_ref_map: HashMap = HashMap::new(); - - // Collect the column references usage in the outer projection. - projection.expr().iter().for_each(|(expr, _)| { - expr.apply(&mut |expr| { - Ok({ - if let Some(column) = expr.as_any().downcast_ref::() { - *column_ref_map.entry(column.clone()).or_default() += 1; - } - VisitRecursion::Continue - }) - }) - .unwrap(); - }); - - // Merging these projections is not beneficial, e.g - // If an expression is not trivial and it is referred more than 1, unifies projections will be - // beneficial as caching mechanism for non-trivial computations. - // See discussion in: https://github.com/apache/arrow-datafusion/issues/8296 - if column_ref_map.iter().any(|(column, count)| { - *count > 1 && !is_expr_trivial(&child.expr()[column.index()].0.clone()) - }) { - return Ok(None); - } - - for (expr, alias) in projection.expr() { - // If there is no match in the input projection, we cannot unify these - // projections. This case will arise if the projection expression contains - // a `PhysicalExpr` variant `update_expr` doesn't support. - let Some(expr) = update_expr(expr, child.expr(), true)? else { - return Ok(None); - }; - projected_exprs.push((expr, alias.clone())); - } - - ProjectionExec::try_new(projected_exprs, child.input().clone()) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Checks if the given expression is trivial. -/// An expression is considered trivial if it is either a `Column` or a `Literal`. -fn is_expr_trivial(expr: &Arc) -> bool { - expr.as_any().downcast_ref::().is_some() - || expr.as_any().downcast_ref::().is_some() -} - -/// Tries to swap `projection` with its input (`output_req`). If possible, -/// performs the swap and returns [`OutputRequirementExec`] as the top plan. -/// Otherwise, returns `None`. -fn try_swapping_with_output_req( - projection: &ProjectionExec, - output_req: &OutputRequirementExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_sort_reqs = vec![]; - // None or empty_vec can be treated in the same way. - if let Some(reqs) = &output_req.required_input_ordering()[0] { - for req in reqs { - let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_sort_reqs.push(PhysicalSortRequirement { - expr: new_expr, - options: req.options, - }); - } - } - - let dist_req = match &output_req.required_input_distribution()[0] { - Distribution::HashPartitioned(exprs) => { - let mut updated_exprs = vec![]; - for expr in exprs { - let Some(new_expr) = update_expr(expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_exprs.push(new_expr); - } - Distribution::HashPartitioned(updated_exprs) - } - dist => dist.clone(), - }; - - make_with_child(projection, &output_req.input()) - .map(|input| { - OutputRequirementExec::new( - input, - (!updated_sort_reqs.is_empty()).then_some(updated_sort_reqs), - dist_req, - ) - }) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Tries to swap `projection` with its input, which is known to be a -/// [`CoalescePartitionsExec`]. If possible, performs the swap and returns -/// [`CoalescePartitionsExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_coalesce_partitions( - projection: &ProjectionExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - // CoalescePartitionsExec always has a single child, so zero indexing is safe. - make_with_child(projection, &projection.input().children()[0]) - .map(|e| Some(Arc::new(CoalescePartitionsExec::new(e)) as _)) -} - -/// Tries to swap `projection` with its input (`filter`). If possible, performs -/// the swap and returns [`FilterExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_filter( - projection: &ProjectionExec, - filter: &FilterExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - // Each column in the predicate expression must exist after the projection. - let Some(new_predicate) = update_expr(filter.predicate(), projection.expr(), false)? - else { - return Ok(None); - }; - - FilterExec::try_new(new_predicate, make_with_child(projection, filter.input())?) - .and_then(|e| { - let selectivity = filter.default_selectivity(); - e.with_default_selectivity(selectivity) - }) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Tries to swap the projection with its input [`RepartitionExec`]. If it can be done, -/// it returns the new swapped version having the [`RepartitionExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_repartition( - projection: &ProjectionExec, - repartition: &RepartitionExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - // If pushdown is not beneficial or applicable, break it. - if projection.benefits_from_input_partitioning()[0] || !all_columns(projection.expr()) - { - return Ok(None); - } - - let new_projection = make_with_child(projection, repartition.input())?; - - let new_partitioning = match repartition.partitioning() { - Partitioning::Hash(partitions, size) => { - let mut new_partitions = vec![]; - for partition in partitions { - let Some(new_partition) = - update_expr(partition, projection.expr(), false)? - else { - return Ok(None); - }; - new_partitions.push(new_partition); - } - Partitioning::Hash(new_partitions, *size) - } - others => others.clone(), - }; - - Ok(Some(Arc::new(RepartitionExec::try_new( - new_projection, - new_partitioning, - )?))) -} - -/// Tries to swap the projection with its input [`SortExec`]. If it can be done, -/// it returns the new swapped version having the [`SortExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sort( - projection: &ProjectionExec, - sort: &SortExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_exprs = vec![]; - for sort in sort.expr() { - let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_exprs.push(PhysicalSortExpr { - expr: new_expr, - options: sort.options, - }); - } - - Ok(Some(Arc::new( - SortExec::new(updated_exprs, make_with_child(projection, sort.input())?) - .with_fetch(sort.fetch()) - .with_preserve_partitioning(sort.preserve_partitioning()), - ))) -} - -/// Tries to swap the projection with its input [`SortPreservingMergeExec`]. -/// If this is possible, it returns the new [`SortPreservingMergeExec`] whose -/// child is a projection. Otherwise, it returns None. -fn try_swapping_with_sort_preserving_merge( - projection: &ProjectionExec, - spm: &SortPreservingMergeExec, -) -> Result>> { - // If the projection does not narrow the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_exprs = vec![]; - for sort in spm.expr() { - let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? - else { - return Ok(None); - }; - updated_exprs.push(PhysicalSortExpr { - expr: updated_expr, - options: sort.options, - }); - } - - Ok(Some(Arc::new( - SortPreservingMergeExec::new( - updated_exprs, - make_with_child(projection, spm.input())?, - ) - .with_fetch(spm.fetch()), - ))) -} - -/// Tries to push `projection` down through `union`. If possible, performs the -/// pushdown and returns a new [`UnionExec`] as the top plan which has projections -/// as its children. Otherwise, returns `None`. -fn try_pushdown_through_union( - projection: &ProjectionExec, - union: &UnionExec, -) -> Result>> { - // If the projection doesn't narrow the schema, we shouldn't try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let new_children = union - .children() - .into_iter() - .map(|child| make_with_child(projection, &child)) - .collect::>>()?; - - Ok(Some(Arc::new(UnionExec::new(new_children)))) -} - -/// Tries to push `projection` down through `hash_join`. If possible, performs the -/// pushdown and returns a new [`HashJoinExec`] as the top plan which has projections -/// as its children. Otherwise, returns `None`. -fn try_pushdown_through_hash_join( - projection: &ProjectionExec, - hash_join: &HashJoinExec, -) -> Result>> { - // Convert projected expressions to columns. We can not proceed if this is - // not possible. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - hash_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - hash_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - hash_join.on(), - ) else { - return Ok(None); - }; - - let new_filter = if let Some(filter) = hash_join.filter() { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - hash_join.left(), - hash_join.right(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - hash_join.left(), - hash_join.right(), - )?; - - Ok(Some(Arc::new(HashJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_on, - new_filter, - hash_join.join_type(), - *hash_join.partition_mode(), - hash_join.null_equals_null, - )?))) -} - -/// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`CrossJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_cross_join( - projection: &ProjectionExec, - cross_join: &CrossJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - cross_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - cross_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - cross_join.left(), - cross_join.right(), - )?; - - Ok(Some(Arc::new(CrossJoinExec::new( - Arc::new(new_left), - Arc::new(new_right), - )))) -} - -/// Tries to swap the projection with its input [`NestedLoopJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`NestedLoopJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_nested_loop_join( - projection: &ProjectionExec, - nl_join: &NestedLoopJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - nl_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - nl_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let new_filter = if let Some(filter) = nl_join.filter() { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - nl_join.left(), - nl_join.right(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - nl_join.left(), - nl_join.right(), - )?; - - Ok(Some(Arc::new(NestedLoopJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_filter, - nl_join.join_type(), - )?))) -} - -/// Tries to swap the projection with its input [`SortMergeJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`SortMergeJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sort_merge_join( - projection: &ProjectionExec, - sm_join: &SortMergeJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - sm_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - sm_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - sm_join.on(), - ) else { - return Ok(None); - }; - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - &sm_join.children()[0], - &sm_join.children()[1], - )?; - - Ok(Some(Arc::new(SortMergeJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_on, - sm_join.filter.clone(), - sm_join.join_type, - sm_join.sort_options.clone(), - sm_join.null_equals_null, - )?))) -} - -/// Tries to swap the projection with its input [`SymmetricHashJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`SymmetricHashJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sym_hash_join( - projection: &ProjectionExec, - sym_join: &SymmetricHashJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - sym_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - sym_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - sym_join.on(), - ) else { - return Ok(None); - }; - - let new_filter = if let Some(filter) = sym_join.filter() { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - sym_join.left(), - sym_join.right(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - sym_join.left(), - sym_join.right(), - )?; - - Ok(Some(Arc::new(SymmetricHashJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_on, - new_filter, - sym_join.join_type(), - sym_join.null_equals_null(), - sym_join.right().output_ordering().map(|p| p.to_vec()), - sym_join.left().output_ordering().map(|p| p.to_vec()), - sym_join.partition_mode(), - )?))) -} - -/// Compare the inputs and outputs of the projection. All expressions must be -/// columns without alias, and projection does not change the order of fields. -fn is_projection_removable(projection: &ProjectionExec) -> bool { - let exprs = projection.expr(); - exprs.iter().enumerate().all(|(idx, (expr, alias))| { - let Some(col) = expr.as_any().downcast_ref::() else { - return false; - }; - col.name() == alias && col.index() == idx - }) && exprs.len() == projection.input().schema().fields().len() -} - -/// Given the expression set of a projection, checks if the projection causes -/// any renaming or constructs a non-`Column` physical expression. -fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|column| column.name() == alias) - .unwrap_or(false) - }) -} - -/// Updates a source provider's projected columns according to the given -/// projection operator's expressions. To use this function safely, one must -/// ensure that all expressions are `Column` expressions without aliases. -fn new_projections_for_columns( - projection: &ProjectionExec, - source: &[usize], -) -> Vec { - projection - .expr() - .iter() - .filter_map(|(expr, _)| { - expr.as_any() - .downcast_ref::() - .map(|expr| source[expr.index()]) - }) - .collect() -} - -/// The function operates in two modes: -/// -/// 1) When `sync_with_child` is `true`: -/// -/// The function updates the indices of `expr` if the expression resides -/// in the input plan. For instance, given the expressions `a@1 + b@2` -/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are -/// updated to `a@0 + b@1` and `c@2`. -/// -/// 2) When `sync_with_child` is `false`: -/// -/// The function determines how the expression would be updated if a projection -/// was placed before the plan associated with the expression. If the expression -/// cannot be rewritten after the projection, it returns `None`. For example, -/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with -/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes -/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. -fn update_expr( - expr: &Arc, - projected_exprs: &[(Arc, String)], - sync_with_child: bool, -) -> Result>> { - #[derive(Debug, PartialEq)] - enum RewriteState { - /// The expression is unchanged. - Unchanged, - /// Some part of the expression has been rewritten - RewrittenValid, - /// Some part of the expression has been rewritten, but some column - /// references could not be. - RewrittenInvalid, - } - - let mut state = RewriteState::Unchanged; - - let new_expr = expr - .clone() - .transform_up_mut(&mut |expr: Arc| { - if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); - } - - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); - }; - if sync_with_child { - state = RewriteState::RewrittenValid; - // Update the index of `column`: - Ok(Transformed::Yes(projected_exprs[column.index()].0.clone())) - } else { - // default to invalid, in case we can't find the relevant column - state = RewriteState::RewrittenInvalid; - // Determine how to update `column` to accommodate `projected_exprs` - projected_exprs - .iter() - .enumerate() - .find_map(|(index, (projected_expr, alias))| { - projected_expr.as_any().downcast_ref::().and_then( - |projected_column| { - (column.name().eq(projected_column.name()) - && column.index() == projected_column.index()) - .then(|| { - state = RewriteState::RewrittenValid; - Arc::new(Column::new(alias, index)) as _ - }) - }, - ) - }) - .map_or_else( - || Ok(Transformed::No(expr)), - |c| Ok(Transformed::Yes(c)), - ) - } - }); - - new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) -} - -/// Creates a new [`ProjectionExec`] instance with the given child plan and -/// projected expressions. -fn make_with_child( - projection: &ProjectionExec, - child: &Arc, -) -> Result> { - ProjectionExec::try_new(projection.expr().to_vec(), child.clone()) - .map(|e| Arc::new(e) as _) -} - -/// Returns `true` if all the expressions in the argument are `Column`s. -fn all_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, _)| expr.as_any().is::()) -} - -/// Downcasts all the expressions in `exprs` to `Column`s. If any of the given -/// expressions is not a `Column`, returns `None`. -fn physical_to_column_exprs( - exprs: &[(Arc, String)], -) -> Option> { - exprs - .iter() - .map(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|col| (col.clone(), alias.clone())) - }) - .collect() -} - -/// Returns the last index before encountering a column coming from the right table when traveling -/// through the projection from left to right, and the last index before encountering a column -/// coming from the left table when traveling through the projection from right to left. -/// If there is no column in the projection coming from the left side, it returns (-1, ...), -/// if there is no column in the projection coming from the right side, it returns (..., projection length). -fn join_table_borders( - left_table_column_count: usize, - projection_as_columns: &[(Column, String)], -) -> (i32, i32) { - let far_right_left_col_ind = projection_as_columns - .iter() - .enumerate() - .take_while(|(_, (projection_column, _))| { - projection_column.index() < left_table_column_count - }) - .last() - .map(|(index, _)| index as i32) - .unwrap_or(-1); - - let far_left_right_col_ind = projection_as_columns - .iter() - .enumerate() - .rev() - .take_while(|(_, (projection_column, _))| { - projection_column.index() >= left_table_column_count - }) - .last() - .map(|(index, _)| index as i32) - .unwrap_or(projection_as_columns.len() as i32); - - (far_right_left_col_ind, far_left_right_col_ind) -} - -/// Tries to update the equi-join `Column`'s of a join as if the the input of -/// the join was replaced by a projection. -fn update_join_on( - proj_left_exprs: &[(Column, String)], - proj_right_exprs: &[(Column, String)], - hash_join_on: &[(PhysicalExprRef, PhysicalExprRef)], -) -> Option> { - // TODO: Clippy wants the "map" call removed, but doing so generates - // a compilation error. Remove the clippy directive once this - // issue is fixed. - #[allow(clippy::map_identity)] - let (left_idx, right_idx): (Vec<_>, Vec<_>) = hash_join_on - .iter() - .map(|(left, right)| (left, right)) - .unzip(); - - let new_left_columns = new_columns_for_join_on(&left_idx, proj_left_exprs); - let new_right_columns = new_columns_for_join_on(&right_idx, proj_right_exprs); - - match (new_left_columns, new_right_columns) { - (Some(left), Some(right)) => Some(left.into_iter().zip(right).collect()), - _ => None, - } -} - -/// This function generates a new set of columns to be used in a hash join -/// operation based on a set of equi-join conditions (`hash_join_on`) and a -/// list of projection expressions (`projection_exprs`). -fn new_columns_for_join_on( - hash_join_on: &[&PhysicalExprRef], - projection_exprs: &[(Column, String)], -) -> Option> { - let new_columns = hash_join_on - .iter() - .filter_map(|on| { - // Rewrite all columns in `on` - (*on) - .clone() - .transform(&|expr| { - if let Some(column) = expr.as_any().downcast_ref::() { - // Find the column in the projection expressions - let new_column = projection_exprs - .iter() - .enumerate() - .find(|(_, (proj_column, _))| { - column.name() == proj_column.name() - }) - .map(|(index, (_, alias))| Column::new(alias, index)); - if let Some(new_column) = new_column { - Ok(Transformed::Yes(Arc::new(new_column))) - } else { - // If the column is not found in the projection expressions, - // it means that the column is not projected. In this case, - // we cannot push the projection down. - Err(DataFusionError::Internal(format!( - "Column {:?} not found in projection expressions", - column - ))) - } - } else { - Ok(Transformed::No(expr)) - } - }) - .ok() - }) - .collect::>(); - (new_columns.len() == hash_join_on.len()).then_some(new_columns) -} - -/// Tries to update the column indices of a [`JoinFilter`] as if the the input of -/// the join was replaced by a projection. -fn update_join_filter( - projection_left_exprs: &[(Column, String)], - projection_right_exprs: &[(Column, String)], - join_filter: &JoinFilter, - join_left: &Arc, - join_right: &Arc, -) -> Option { - let mut new_left_indices = new_indices_for_join_filter( - join_filter, - JoinSide::Left, - projection_left_exprs, - join_left.schema(), - ) - .into_iter(); - let mut new_right_indices = new_indices_for_join_filter( - join_filter, - JoinSide::Right, - projection_right_exprs, - join_right.schema(), - ) - .into_iter(); - - // Check if all columns match: - (new_right_indices.len() + new_left_indices.len() - == join_filter.column_indices().len()) - .then(|| { - JoinFilter::new( - join_filter.expression().clone(), - join_filter - .column_indices() - .iter() - .map(|col_idx| ColumnIndex { - index: if col_idx.side == JoinSide::Left { - new_left_indices.next().unwrap() - } else { - new_right_indices.next().unwrap() - }, - side: col_idx.side, - }) - .collect(), - join_filter.schema().clone(), - ) - }) -} - -/// This function determines and returns a vector of indices representing the -/// positions of columns in `projection_exprs` that are involved in `join_filter`, -/// and correspond to a particular side (`join_side`) of the join operation. -fn new_indices_for_join_filter( - join_filter: &JoinFilter, - join_side: JoinSide, - projection_exprs: &[(Column, String)], - join_child_schema: SchemaRef, -) -> Vec { - join_filter - .column_indices() - .iter() - .filter(|col_idx| col_idx.side == join_side) - .filter_map(|col_idx| { - projection_exprs.iter().position(|(col, _)| { - col.name() == join_child_schema.fields()[col_idx.index].name() - }) - }) - .collect() -} - -/// Checks three conditions for pushing a projection down through a join: -/// - Projection must narrow the join output schema. -/// - Columns coming from left/right tables must be collected at the left/right -/// sides of the output table. -/// - Left or right table is not lost after the projection. -fn join_allows_pushdown( - projection_as_columns: &[(Column, String)], - join_schema: SchemaRef, - far_right_left_col_ind: i32, - far_left_right_col_ind: i32, -) -> bool { - // Projection must narrow the join output: - projection_as_columns.len() < join_schema.fields().len() - // Are the columns from different tables mixed? - && (far_right_left_col_ind + 1 == far_left_right_col_ind) - // Left or right table is not lost after the projection. - && far_right_left_col_ind >= 0 - && far_left_right_col_ind < projection_as_columns.len() as i32 -} - -/// If pushing down the projection over this join's children seems possible, -/// this function constructs the new [`ProjectionExec`]s that will come on top -/// of the original children of the join. -fn new_join_children( - projection_as_columns: Vec<(Column, String)>, - far_right_left_col_ind: i32, - far_left_right_col_ind: i32, - left_child: &Arc, - right_child: &Arc, -) -> Result<(ProjectionExec, ProjectionExec)> { - let new_left = ProjectionExec::try_new( - projection_as_columns[0..=far_right_left_col_ind as _] - .iter() - .map(|(col, alias)| { - ( - Arc::new(Column::new(col.name(), col.index())) as _, - alias.clone(), - ) - }) - .collect_vec(), - left_child.clone(), - )?; - let left_size = left_child.schema().fields().len() as i32; - let new_right = ProjectionExec::try_new( - projection_as_columns[far_left_right_col_ind as _..] - .iter() - .map(|(col, alias)| { - ( - Arc::new(Column::new( - col.name(), - // Align projected expressions coming from the right - // table with the new right child projection: - (col.index() as i32 - left_size) as _, - )) as _, - alias.clone(), - ) - }) - .collect_vec(), - right_child.clone(), - )?; - - Ok((new_left, new_right)) -} - -#[cfg(test)] -mod tests { - use std::sync::Arc; - - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; - use crate::physical_optimizer::output_requirements::OutputRequirementExec; - use crate::physical_optimizer::projection_pushdown::{ - join_table_borders, update_expr, ProjectionPushdown, - }; - use crate::physical_optimizer::PhysicalOptimizerRule; - use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::filter::FilterExec; - use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; - use crate::physical_plan::joins::StreamJoinPartitionMode; - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::projection::ProjectionExec; - use crate::physical_plan::repartition::RepartitionExec; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - use crate::physical_plan::{get_plan_string, ExecutionPlan}; - - use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; - use datafusion_common::config::ConfigOptions; - use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; - use datafusion_execution::object_store::ObjectStoreUrl; - use datafusion_execution::{SendableRecordBatchStream, TaskContext}; - use datafusion_expr::{ColumnarValue, Operator}; - use datafusion_physical_expr::expressions::{ - BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, - }; - use datafusion_physical_expr::{ - Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, ScalarFunctionExpr, - }; - use datafusion_physical_plan::joins::SymmetricHashJoinExec; - use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; - use datafusion_physical_plan::union::UnionExec; - - use itertools::Itertools; - - #[test] - fn test_update_matching_exprs() -> Result<()> { - let exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Divide, - Arc::new(Column::new("e", 5)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 3)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 0)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - None, - false, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 2))), - vec![ - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 2)), - Operator::Plus, - Arc::new(Column::new("e", 5)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 5)), - Operator::Plus, - Arc::new(Column::new("d", 2)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Modulo, - Arc::new(Column::new("e", 5)), - ))), - )?), - ]; - let child: Vec<(Arc, String)> = vec![ - (Arc::new(Column::new("c", 2)), "c".to_owned()), - (Arc::new(Column::new("b", 1)), "b".to_owned()), - (Arc::new(Column::new("d", 3)), "d".to_owned()), - (Arc::new(Column::new("a", 0)), "a".to_owned()), - (Arc::new(Column::new("f", 5)), "f".to_owned()), - (Arc::new(Column::new("e", 4)), "e".to_owned()), - ]; - - let expected_exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Divide, - Arc::new(Column::new("e", 4)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 0)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 2)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - None, - false, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 3))), - vec![ - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 3)), - Operator::Plus, - Arc::new(Column::new("e", 4)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 4)), - Operator::Plus, - Arc::new(Column::new("d", 3)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Modulo, - Arc::new(Column::new("e", 4)), - ))), - )?), - ]; - - for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { - assert!(update_expr(&expr, &child, true)? - .unwrap() - .eq(&expected_expr)); - } - - Ok(()) - } - - #[test] - fn test_update_projected_exprs() -> Result<()> { - let exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Divide, - Arc::new(Column::new("e", 5)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 3)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 0)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - None, - false, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 2))), - vec![ - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 2)), - Operator::Plus, - Arc::new(Column::new("e", 5)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 5)), - Operator::Plus, - Arc::new(Column::new("d", 2)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Modulo, - Arc::new(Column::new("e", 5)), - ))), - )?), - ]; - let projected_exprs: Vec<(Arc, String)> = vec![ - (Arc::new(Column::new("a", 3)), "a".to_owned()), - (Arc::new(Column::new("b", 1)), "b_new".to_owned()), - (Arc::new(Column::new("c", 0)), "c".to_owned()), - (Arc::new(Column::new("d", 2)), "d_new".to_owned()), - (Arc::new(Column::new("e", 5)), "e".to_owned()), - (Arc::new(Column::new("f", 4)), "f_new".to_owned()), - ]; - - let expected_exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Divide, - Arc::new(Column::new("e", 4)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 0)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_new", 1)), - Operator::Divide, - Arc::new(Column::new("c", 2)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Divide, - Arc::new(Column::new("b_new", 1)), - )), - ], - DataType::Int32, - None, - false, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d_new", 3))), - vec![ - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d_new", 3)), - Operator::Plus, - Arc::new(Column::new("e", 4)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 4)), - Operator::Plus, - Arc::new(Column::new("d_new", 3)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Modulo, - Arc::new(Column::new("e", 4)), - ))), - )?), - ]; - - for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { - assert!(update_expr(&expr, &projected_exprs, false)? - .unwrap() - .eq(&expected_expr)); - } - - Ok(()) - } - - #[test] - fn test_join_table_borders() -> Result<()> { - let projections = vec![ - (Column::new("b", 1), "b".to_owned()), - (Column::new("c", 2), "c".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("d", 3), "d".to_owned()), - (Column::new("c", 2), "c".to_owned()), - (Column::new("f", 5), "f".to_owned()), - (Column::new("h", 7), "h".to_owned()), - (Column::new("g", 6), "g".to_owned()), - ]; - let left_table_column_count = 5; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (4, 5) - ); - - let left_table_column_count = 8; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (7, 8) - ); - - let left_table_column_count = 1; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (-1, 0) - ); - - let projections = vec![ - (Column::new("a", 0), "a".to_owned()), - (Column::new("b", 1), "b".to_owned()), - (Column::new("d", 3), "d".to_owned()), - (Column::new("g", 6), "g".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("f", 5), "f".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("h", 7), "h".to_owned()), - ]; - let left_table_column_count = 5; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (2, 7) - ); - - let left_table_column_count = 7; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (6, 7) - ); - - Ok(()) - } - - fn create_simple_csv_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])); - Arc::new(CsvExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema.clone(), - file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_unknown(&schema), - projection: Some(vec![0, 1, 2, 3, 4]), - limit: None, - table_partition_cols: vec![], - output_ordering: vec![vec![]], - }, - false, - 0, - 0, - None, - FileCompressionType::UNCOMPRESSED, - )) - } - - fn create_projecting_csv_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - ])); - Arc::new(CsvExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema.clone(), - file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_unknown(&schema), - projection: Some(vec![3, 2, 1]), - limit: None, - table_partition_cols: vec![], - output_ordering: vec![vec![]], - }, - false, - 0, - 0, - None, - FileCompressionType::UNCOMPRESSED, - )) - } - - fn create_projecting_memory_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])); - - Arc::new(MemoryExec::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap()) - } - - #[test] - fn test_csv_after_projection() -> Result<()> { - let csv = create_projecting_csv_exec(); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 2)), "b".to_string()), - (Arc::new(Column::new("d", 0)), "d".to_string()), - ], - csv.clone(), - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@2 as b, d@0 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[d, c, b], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "CsvExec: file_groups={1 group: [[x]]}, projection=[b, d], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_memory_after_projection() -> Result<()> { - let memory = create_projecting_memory_exec(); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("d", 2)), "d".to_string()), - (Arc::new(Column::new("e", 3)), "e".to_string()), - (Arc::new(Column::new("a", 1)), "a".to_string()), - ], - memory.clone(), - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[d@2 as d, e@3 as e, a@1 as a]", - " MemoryExec: partitions=0, partition_sizes=[]", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = ["MemoryExec: partitions=0, partition_sizes=[]"]; - assert_eq!(get_plan_string(&after_optimize), expected); - assert_eq!( - after_optimize - .clone() - .as_any() - .downcast_ref::() - .unwrap() - .projection() - .clone() - .unwrap(), - vec![3, 4, 0] - ); - - Ok(()) - } - - #[test] - fn test_streaming_table_after_projection() -> Result<()> { - struct DummyStreamPartition { - schema: SchemaRef, - } - impl PartitionStream for DummyStreamPartition { - fn schema(&self) -> &SchemaRef { - &self.schema - } - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - unreachable!() - } - } - - let streaming_table = StreamingTableExec::try_new( - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])), - vec![Arc::new(DummyStreamPartition { - schema: Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])), - }) as _], - Some(&vec![0_usize, 2, 4, 3]), - vec![ - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("e", 2)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }, - ], - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options: SortOptions::default(), - }], - ] - .into_iter(), - true, - )?; - let projection = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("d", 3)), "d".to_string()), - (Arc::new(Column::new("e", 2)), "e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - ], - Arc::new(streaming_table) as _, - )?) as _; - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let result = after_optimize - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!( - result.partition_schema(), - &Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])) - ); - assert_eq!( - result.projection().clone().unwrap().to_vec(), - vec![3_usize, 4, 0] - ); - assert_eq!( - result.projected_schema(), - &Schema::new(vec![ - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - Field::new("a", DataType::Int32, true), - ]) - ); - assert_eq!( - result.projected_output_ordering().into_iter().collect_vec(), - vec![ - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("e", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 2)), - options: SortOptions::default(), - }, - ], - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("d", 0)), - options: SortOptions::default(), - }], - ] - ); - assert!(result.is_infinite()); - - Ok(()) - } - - #[test] - fn test_projection_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let child_projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("e", 4)), "new_e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("b", 1)), "new_b".to_string()), - ], - csv.clone(), - )?); - let top_projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("new_b", 3)), "new_b".to_string()), - ( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_e", 1)), - )), - "binary".to_string(), - ), - (Arc::new(Column::new("new_b", 3)), "newest_b".to_string()), - ], - child_projection.clone(), - )?); - - let initial = get_plan_string(&top_projection); - let expected_initial = [ - "ProjectionExec: expr=[new_b@3 as new_b, c@0 + new_e@1 as binary, new_b@3 as newest_b]", - " ProjectionExec: expr=[c@2 as c, e@4 as new_e, a@0 as a, b@1 as new_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(top_projection, &ConfigOptions::new())?; - - let expected = [ - "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_output_req_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(OutputRequirementExec::new( - csv.clone(), - Some(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 1)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: Some(SortOptions::default()), - }, - ]), - Distribution::HashPartitioned(vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " OutputRequirementExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected: [&str; 3] = [ - "OutputRequirementExec", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - - assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 2)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_a", 1)), - )), - options: Some(SortOptions::default()), - }, - ]; - assert_eq!( - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .required_input_ordering()[0] - .clone() - .unwrap(), - expected_reqs - ); - let expected_distribution: Vec> = vec![ - Arc::new(Column::new("new_a", 1)), - Arc::new(Column::new("b", 2)), - ]; - if let Distribution::HashPartitioned(vec) = after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .required_input_distribution()[0] - .clone() - { - assert!(vec - .iter() - .zip(expected_distribution) - .all(|(actual, expected)| actual.eq(&expected))); - } else { - panic!("Expected HashPartitioned distribution!"); - }; - - Ok(()) - } - - #[test] - fn test_coalesce_partitions_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let coalesce_partitions: Arc = - Arc::new(CoalescePartitionsExec::new(csv)); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - ], - coalesce_partitions, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", - " CoalescePartitionsExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "CoalescePartitionsExec", - " ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_filter_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let predicate = Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Minus, - Arc::new(Column::new("a", 0)), - )), - Operator::Gt, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 3)), - Operator::Minus, - Arc::new(Column::new("a", 0)), - )), - )); - let filter: Arc = - Arc::new(FilterExec::try_new(predicate, csv)?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - ], - filter.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", - " FilterExec: b@1 - a@0 > d@3 - a@0", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "FilterExec: b@1 - a_new@0 > d@2 - a_new@0", - " ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_join_after_projection() -> Result<()> { - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( - left_csv, - right_csv, - vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], - // b_left-(1+a_right)<=a_right+c_left - Some(JoinFilter::new( - Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_left_inter", 0)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a_right_inter", 1)), - )), - )), - Operator::LtEq, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a_right_inter", 1)), - Operator::Plus, - Arc::new(Column::new("c_left_inter", 2)), - )), - )), - vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 2, - side: JoinSide::Left, - }, - ], - Schema::new(vec![ - Field::new("b_left_inter", DataType::Int32, true), - Field::new("a_right_inter", DataType::Int32, true), - Field::new("c_left_inter", DataType::Int32, true), - ]), - )), - &JoinType::Inner, - true, - None, - None, - StreamJoinPartitionMode::SinglePartition, - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), - (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), - (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), - (Arc::new(Column::new("a", 5)), "a_from_right".to_string()), - (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), - ], - join, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b_from_left@1, c_from_right@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[a@0 as a_from_right, c@2 as c_from_right]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - let expected_filter_col_ind = vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 0, - side: JoinSide::Left, - }, - ]; - - assert_eq!( - expected_filter_col_ind, - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .filter() - .unwrap() - .column_indices() - ); - - Ok(()) - } - - #[test] - fn test_join_after_required_projection() -> Result<()> { - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( - left_csv, - right_csv, - vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], - // b_left-(1+a_right)<=a_right+c_left - Some(JoinFilter::new( - Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_left_inter", 0)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a_right_inter", 1)), - )), - )), - Operator::LtEq, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a_right_inter", 1)), - Operator::Plus, - Arc::new(Column::new("c_left_inter", 2)), - )), - )), - vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 2, - side: JoinSide::Left, - }, - ], - Schema::new(vec![ - Field::new("b_left_inter", DataType::Int32, true), - Field::new("a_right_inter", DataType::Int32, true), - Field::new("c_left_inter", DataType::Int32, true), - ]), - )), - &JoinType::Inner, - true, - None, - None, - StreamJoinPartitionMode::SinglePartition, - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("a", 5)), "a".to_string()), - (Arc::new(Column::new("b", 6)), "b".to_string()), - (Arc::new(Column::new("c", 7)), "c".to_string()), - (Arc::new(Column::new("d", 8)), "d".to_string()), - (Arc::new(Column::new("e", 9)), "e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - (Arc::new(Column::new("e", 4)), "e".to_string()), - ], - join, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - Ok(()) - } - - #[test] - fn test_repartition_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let repartition: Arc = Arc::new(RepartitionExec::try_new( - csv, - Partitioning::Hash( - vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("d", 3)), - ], - 6, - ), - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("d", 3)), "d_new".to_string()), - ], - repartition, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", - " RepartitionExec: partitioning=Hash([a@0, b@1, d@3], 6), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "RepartitionExec: partitioning=Hash([a@1, b_new@0, d_new@2], 6), input_partitions=1", - " ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - assert_eq!( - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .partitioning() - .clone(), - Partitioning::Hash( - vec![ - Arc::new(Column::new("a", 1)), - Arc::new(Column::new("b_new", 0)), - Arc::new(Column::new("d_new", 2)), - ], - 6, - ), - ); - - Ok(()) - } - - #[test] - fn test_sort_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(SortExec::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: SortOptions::default(), - }, - ], - csv.clone(), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SortExec: expr=[b@2 ASC,c@0 + new_a@1 ASC]", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_sort_preserving_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: SortOptions::default(), - }, - ], - csv.clone(), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SortPreservingMergeExec: [b@2 ASC,c@0 + new_a@1 ASC]", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_union_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let union: Arc = - Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - union.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " UnionExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "UnionExec", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index dd0261420304..278a8895164d 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -70,7 +70,7 @@ impl TopKAggregation { // We found what we want: clone, copy the limit down, and return modified node let new_aggr = AggregateExec::try_new( *aggr.mode(), - aggr.group_by().clone(), + aggr.group_expr().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), aggr.input().clone(), diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index a1d9a02cf6b1..fa160f2a117e 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -656,18 +656,18 @@ async fn test_physical_plan_display_indent_multi_children() { let dataframe = ctx.sql(sql).await.unwrap(); let physical_plan = dataframe.create_physical_plan().await.unwrap(); let expected = vec![ - "ProjectionExec: expr=[c1@0 as c1]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + "CoalesceBatchesExec: target_batch_size=4096", + " ProjectionExec: expr=[c1@0 as c1]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", + " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=9000", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " ProjectionExec: expr=[c1@0 as c2]", - " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=9000", + " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + " ProjectionExec: expr=[c1@0 as c2]", + " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true" ]; let normalizer = ExplainNormalizer::new(); diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 2bb205ce90dc..c4a1aa7fbb5f 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -133,6 +133,15 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { fn create_sliding_accumulator(&self) -> Result> { not_impl_err!("Retractable Accumulator hasn't been implemented for {self:?} yet") } + + /// Rewrites the aggregate expression with the given expressions. + /// The order of the given expressions is taken into account while replacing. + fn with_new_expressions( + self: Arc, + expressions: Vec>, + ) -> Option> { + None + } } /// Checks whether the given aggregate expression is order-sensitive. diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index 6cf2810ce588..9b42cca76bd0 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -153,6 +153,19 @@ impl AggregateExpr for Sum { } downcast_sum!(self, helper) } + + fn with_new_expressions( + self: Arc, + mut expressions: Vec>, + ) -> Option> { + Some(Arc::new(Self { + name: self.name.clone(), + data_type: self.data_type.clone(), + return_type: self.return_type.clone(), + expr: expressions.swap_remove(0), + nullable: self.nullable, + })) + } } impl PartialEq for Sum { diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index e2714dc42bea..78b653a47b7e 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -27,7 +27,9 @@ use arrow::compute::kernels::sort::SortColumn; use arrow::compute::SortOptions; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; -use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{ + internal_err, not_impl_err, DataFusionError, Result, ScalarValue, +}; use datafusion_expr::window_state::{ PartitionBatchState, WindowAggState, WindowFrameContext, }; @@ -127,6 +129,15 @@ pub trait WindowExpr: Send + Sync + Debug { /// Get the reverse expression of this [WindowExpr]. fn get_reverse_expr(&self) -> Option>; + + /// Rewrites the window expression with the given expressions. + /// The order of the given expressions is taken into account while replacing. + fn with_new_expressions( + self: Arc, + expressions: Vec>, + ) -> Option> { + None + } } /// Extension trait that adds common functionality to [`AggregateWindowExpr`]s diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 156362430558..c116d4b96157 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -471,17 +471,13 @@ impl AggregateExec { } } - pub fn group_by(&self) -> &PhysicalGroupBy { - &self.group_by - } - /// true, if this Aggregate has a group-by with no required or explicit ordering, /// no filtering and no aggregate expressions /// This method qualifies the use of the LimitedDistinctAggregation rewrite rule /// on an AggregateExec. pub fn is_unordered_unfiltered_group_by_distinct(&self) -> bool { // ensure there is a group by - if self.group_by().is_empty() { + if self.group_expr().is_empty() { return false; } // ensure there are no aggregate expressions diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 1c4a6ac0ecaf..675b003cac4b 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -39,6 +39,7 @@ use datafusion_physical_expr::{ }; use futures::stream::TryStreamExt; +use projection::ProjectionExec; use tokio::task::JoinSet; mod topk; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index cc2ab62049ed..6c3d21955607 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -36,6 +36,7 @@ use crate::{ use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; @@ -66,11 +67,33 @@ pub struct ProjectionExec { impl ProjectionExec { /// Create a projection on an input pub fn try_new( - expr: Vec<(Arc, String)>, + mut expr: Vec<(Arc, String)>, input: Arc, ) -> Result { let input_schema = input.schema(); - + expr = expr + .iter() + .enumerate() + .map(|(expr_idx, (expression, name))| { + expression + .clone() + .transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => { + // Sometimes, an expression and its name in the input_schema + // doesn't match. This can cause problems, so we make sure + // that the expression name matches with the name in `input_schema`. + // Conceptually, `source_expr` and `expression` should be the same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + }) + .map(|source_expr| (source_expr, name.to_string())) + }) + .collect::>>()?; let fields: Result> = expr .iter() .map(|(e, name)| { @@ -94,7 +117,6 @@ impl ProjectionExec { // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; - let input_eqs = input.equivalence_properties(); let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); let output_ordering = project_eqs.oeq_class().output_ordering(); diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 6b9db5589391..172db0bc0570 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -137,7 +137,8 @@ Sort: balances.time ASC NULLS LAST, balances.name ASC NULLS LAST, balances.accou physical_plan SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST] --RecursiveQueryExec: name=balances, is_distinct=false -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true +----ProjectionExec: expr=[time@0 as time, name@1 as name, account_balance@2 as account_balance] +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true ----CoalescePartitionsExec ------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] --------CoalesceBatchesExec: target_batch_size=2 diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 3f609e254839..d897d7c733d4 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -94,7 +94,7 @@ Projection: FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 A ----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]]] ------TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan -ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] +ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3)@1 as c3, FIRST_VALUE(aggregate_test_100.c2)@2 as c2] --SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ----SortExec: expr=[c1@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 4002164cc918..d9f9fde45e93 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,8 +43,8 @@ Projection: aggregate_test_100.c1 --Filter: aggregate_test_100.c2 > Int8(10) ----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -ProjectionExec: expr=[c1@0 as c1] ---CoalesceBatchesExec: target_batch_size=8192 +CoalesceBatchesExec: target_batch_size=8192 +--ProjectionExec: expr=[c1@0 as c1] ----FilterExec: c2@1 > 10 ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true @@ -252,7 +252,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 PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after OptimizeProjections 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]:)]] @@ -309,7 +309,7 @@ GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Co --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 PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after OptimizeProjections SAME TEXT AS ABOVE physical_plan 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]:)]] --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]:)]] @@ -344,7 +344,7 @@ GlobalLimitExec: skip=0, fetch=10 --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 PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after OptimizeProjections SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10 --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/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 1312f2916ed6..84c884b86716 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -57,8 +57,8 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 -----ProjectionExec: expr=[a@1 as a] -------CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=8192 +------ProjectionExec: expr=[a@1 as a] --------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true ----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -102,12 +102,13 @@ GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [a2@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 ----ProjectionExec: expr=[a@0 as a2, b@1 as b] ------CoalesceBatchesExec: target_batch_size=8192 ---------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true -----------CoalesceBatchesExec: target_batch_size=8192 -------------FilterExec: d@3 = 3 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +--------ProjectionExec: expr=[a@0 as a, b@1 as b] +----------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true +------------CoalesceBatchesExec: target_batch_size=8192 +--------------FilterExec: d@3 = 3 +----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index e063d6e8960a..3748253be046 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -370,9 +370,9 @@ AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------ProjectionExec: expr=[] -----------GlobalLimitExec: skip=6, fetch=3 -------------CoalesceBatchesExec: target_batch_size=8192 +--------GlobalLimitExec: skip=6, fetch=3 +----------CoalesceBatchesExec: target_batch_size=8192 +------------ProjectionExec: expr=[] --------------FilterExec: a@0 > 3 ----------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index ba407f6d2852..8a05e49192fd 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -635,8 +635,8 @@ Projection: lineitem.l_partkey ----Filter: (part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) ------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)] physical_plan -ProjectionExec: expr=[l_partkey@0 as l_partkey] ---CoalesceBatchesExec: target_batch_size=8192 +CoalesceBatchesExec: target_batch_size=8192 +--ProjectionExec: expr=[l_partkey@0 as l_partkey] ----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15 ------CoalesceBatchesExec: target_batch_size=8192 --------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 @@ -729,11 +729,11 @@ Aggregate: groupBy=[[part.p_partkey]], aggr=[[SUM(lineitem.l_extendedprice), AVG ------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] ---ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, p_partkey@2 as p_partkey, ps_suppkey@4 as ps_suppkey] -----CoalesceBatchesExec: target_batch_size=8192 +--CoalesceBatchesExec: target_batch_size=8192 +----ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, p_partkey@2 as p_partkey, ps_suppkey@4 as ps_suppkey] ------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)] ---------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey] -----------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=8192 +----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey] ------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 @@ -741,8 +741,8 @@ AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM --------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ---------------------CoalesceBatchesExec: target_batch_size=8192 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ----------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 ------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index b7bbc0706576..0b0b77f0af5e 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1364,6 +1364,7 @@ AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[COUNT(*)] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2], has_header=true + statement ok drop table annotated_data_finite2; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index fa4445d4cd4c..fbe2b847a54a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1752,8 +1752,8 @@ ProjectionExec: expr=[COUNT(*)@0 as global_count] --------------CoalesceBatchesExec: target_batch_size=4096 ----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] ---------------------ProjectionExec: expr=[c1@0 as c1] -----------------------CoalesceBatchesExec: target_batch_size=4096 +--------------------CoalesceBatchesExec: target_batch_size=4096 +----------------------ProjectionExec: expr=[c1@0 as c1] ------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 --------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true @@ -2814,13 +2814,13 @@ Projection: sum1, sum2, count1, count2 ------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col --------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] -----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] -------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +GlobalLimitExec: skip=0, fetch=5 +--ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] +----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] +--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] +----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] + query IIII SELECT @@ -2861,14 +2861,12 @@ Projection: sum1, sum2, count1, count2 ------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col --------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] -----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] -------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] - +GlobalLimitExec: skip=0, fetch=5 +--ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] +----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] +--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] +----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT