diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index fd471e750194..ae56673608e1 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1366,6 +1366,7 @@ dependencies = [ "arrow", "datafusion-common", "datafusion-expr", + "indexmap 2.2.6", ] [[package]] diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index 62537169cfc6..dbb3ea853017 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -113,7 +113,7 @@ fn get_common_requirement_of_aggregate_input( InputOrderMode::Linear }; let projection_mapping = - ProjectionMapping::try_new(group_by.expr(), &input.schema())?; + ProjectionMapping::try_new(group_by.expr().to_vec(), &input.schema())?; let cache = AggregateExec::compute_properties( input, diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index c80668c6da74..9ca8f1f6992f 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -29,10 +29,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..764a82e99b73 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -0,0 +1,6289 @@ +// 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 to achieve 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 the root. +//! 2. Ensuring projections are used only when they contribute to narrowing the schema, +//! or when necessary for evaluation or aliasing purposes. +//! +//! The optimization works in two phases: +//! +//! Top-down Phase: +//! --------------- +//! - Traverses the plan from the root to leaves. If the node is: +//! 1. A projection node, it may: +//! a. Merge it with its input projection if beneficial. +//! b. Remove the projection if it is redundant. +//! c. Narrow the projection if possible. +//! d. Nest the projection into the input. +//! e. Do nothing. +//! 2. A non-projection node, it may +//! a. Insert the necessary projection into input if the schema needs pruning. +//! b. Do nothing if all fields are necessary. +//! +//! Bottom-up Phase: +//! ---------------- +//! This phase is necessary because modifying a plan node in the top-down phase +//! can require a change in column indices used by downstream 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 column indices +//! in the plans by referencing these mapping records. Furthermore, in certain +//! cases, we can only deduce that a projection is unnecessary after completing +//! the top-down traversal. We remove such projections in the bottom-up phase +//! by checking their input schema mapping. +//! The code implementing the bottom-up phase resides in the `with_new_children` +//! implementation of the `ConcreteTreeNode` trait. + +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 datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{ + ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, +}; +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::window::WindowExpr; +use datafusion_physical_expr::{ + AggregateExpr, ExprMapping, LexOrdering, Partitioning, PhysicalExpr, PhysicalExprRef, + PhysicalSortExpr, +}; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::insert::DataSinkExec; +use datafusion_physical_plan::joins::utils::{ + ColumnIndex, JoinFilter, JoinOn, JoinOnRef, +}; +use datafusion_physical_plan::joins::{ + CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, + SymmetricHashJoinExec, +}; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::recursive_query::RecursiveQueryExec; +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 indexmap::{IndexMap, IndexSet}; +use itertools::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 { + /// The plan resides in the node + pub plan: Arc, + /// The node above expects it can reach these columns. + pub required_columns: IndexSet, + /// The nodes above will be updated according to these matches. First element indicates + /// the initial column index, and the second element is for the updated version. + pub schema_mapping: IndexMap, + /// Children nodes + 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, +/// just like `required_columns` field in [`ProjectionOptimizer`]. +type ColumnRequirements = IndexMap; + +impl ProjectionOptimizer { + /// Constructs the empty tree according to the plan. All state information is empty initially. + fn new_default(plan: Arc) -> Self { + let children = plan.children(); + Self { + plan, + required_columns: IndexSet::new(), + schema_mapping: IndexMap::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 { + // If the node is a source provider, no need a change. + if self.children_nodes.is_empty() { + // We also clean the requirements, since we would like + // to leave payload-free nodes after the rule finishes. + self.required_columns.clear(); + return Ok(self); + } + + if self.plan.as_any().is::() { + // If the node is a projection, it is analyzed and may be rewritten + // to make the projection more efficient, or even it may be removed. + self.optimize_projections() + } else { + // If the node is 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 swapped with or embedded into the plan below. + /// If none of them is possible, it remains unchanged. + fn optimize_projections(mut self) -> Result { + let projection_input = &self.plan.children()[0]; + + // We first need to check having 2 sequential projections in case of merging them. + if projection_input.as_any().is::() { + self = match self.try_unify_projections()? { + unified_plans if unified_plans.transformed => { + // 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.data.optimize_projections(); + } + no_change => no_change.data, + }; + } + + // These kind of plans can swap the order with projections without any further modification. + if is_plan_schema_agnostic(projection_input) { + self = match self.try_swap_trivial()? { + swapped if swapped.transformed => { + return Ok(swapped.data); + } + no_change => no_change.data, + }; + } + + // The projection can be removed. To avoid making unnecessary operations, + // try_remove should be called before try_narrow. + self = match self.try_remove_projection()? { + removed if removed.transformed => { + // 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.data.adjust_node_with_requirements(); + } + no_change => no_change.data, + }; + + // The projection can get narrower. + self = match self.try_narrow_projection()? { + narrowed if narrowed.transformed => { + // We need to re-run the rule on the new node since it may be removed within the plan below. + return narrowed.data.optimize_projections(); + } + no_change => no_change.data, + }; + + // HashJoinExec can own the projection above. + if projection_input.as_any().is::() { + return match self.try_embed_to_hash_join()? { + join if join.transformed => { + // Re-run on the new HashJoin node + join.data.adjust_node_with_requirements() + } + projection => { + let mut node = projection.data; + node.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&node.children_nodes[0].plan); + Ok(node) + } + }; + } + + // Source providers: + // NOTE: No need to handle source providers separately since if they have projected + // any unnecessary columns, a projection appears on top of them. + if projection_input.as_any().is::() { + return self.try_projected_csv(); + } + // TODO: Other source execs can be implemented here if projection can be applied inside them. + + // 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. + self.children_nodes[0].required_columns = if let Some(projection_plan) = + self.plan.as_any().downcast_ref::() + { + collect_projection_input_requirements( + mem::take(&mut self.required_columns), + projection_plan.expr(), + ) + } else { + // If the method is used with a non-projection plan, we must sustain the execution safely. + collect_columns_in_plan_schema(projection_input) + }; + + Ok(self) + } + + /// If it is beneficial, unifies the projection and its input, which is also a [`ProjectionExec`]. + fn try_unify_projections(mut self) -> Result> { + // These are known to be a ProjectionExec. + let Some(projection) = self.plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(self)); + }; + let Some(child_projection) = self.children_nodes[0] + .plan + .as_any() + .downcast_ref::() + else { + return Ok(Transformed::no(self)); + }; + // Projection can be beneficial if it caches any computation which are used more than once. + if caching_projections(projection, child_projection)? { + return Ok(Transformed::no(self)); + } + + // Create the new projection expression by indexing the parent + // projection expressions according to the input projection expressions. + let mut projected_exprs = vec![]; + for (expr, alias) in projection.expr() { + let Some(expr) = + update_expr_with_projection(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(Arc::new)?; + + Ok(Transformed::yes(Self { + 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) -> Result> { + // It must be a projection. + let Some(projection_exec) = self.plan.as_any().downcast_ref::() + else { + return Ok(Transformed::no(self)); + }; + + // The projection must have all column expressions without aliases. + let Some(projection_columns) = + try_collect_alias_free_columns(projection_exec.expr()) + else { + return Ok(Transformed::no(self)); + }; + + // Input requirements of the projection in terms of projection's parent requirements: + let projection_requires = + map_parent_reqs_to_input_reqs(&self.required_columns, &projection_columns); + let input_columns = collect_columns_in_plan_schema(projection_exec.input()); + // If all fields of the input are necessary, we can remove the projection. + if all_input_columns_required(&input_columns, &projection_requires) { + let schema_mapping = index_changes_after_projection_removal( + self.required_columns, + &projection_columns, + ); + let new_current_node = self.children_nodes.swap_remove(0); + + Ok(Transformed::yes(ProjectionOptimizer { + plan: new_current_node.plan, + required_columns: projection_requires, + schema_mapping, + children_nodes: new_current_node.children_nodes, + })) + } else { + Ok(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 Some(projection_exec) = self.plan.as_any().downcast_ref::() + else { + return Ok(Transformed::no(self)); + }; + + let requirement_map = analyze_requirements(&self); + let (used_columns, unused_columns) = + partition_column_requirements(requirement_map); + + // If all projected items are used, we cannot get a narrower projection. + if unused_columns.is_empty() { + return Ok(Transformed::no(self)); + } + + let projected_exprs = collect_used_columns(projection_exec.expr(), &used_columns); + let new_mapping = + calculate_column_mapping(&self.required_columns, &unused_columns); + + let new_projection_plan = Arc::new(ProjectionExec::try_new( + projected_exprs, + self.children_nodes[0].plan.clone(), + )?); + + // The requirements of the node are updated according to the new projection schema. + let new_projection_requires = self + .required_columns + .into_iter() + .map(|col| new_mapping.get(&col).cloned().unwrap_or(col)) + .collect(); + + Ok(Transformed::yes(ProjectionOptimizer { + plan: new_projection_plan, + required_columns: new_projection_requires, + schema_mapping: new_mapping, + children_nodes: self.children_nodes, + })) + } + + /// By embedding projections to hash joins, we can reduce the cost of `build_batch_from_indices` + /// in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid + /// unnecessary output creation. + fn try_embed_to_hash_join(mut self) -> Result> { + // These are known. + let Some(projection) = self.plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(self)); + }; + let Some(hash_join) = projection.input().as_any().downcast_ref::() + else { + return Ok(Transformed::no(self)); + }; + + // Collect all column indices from the given projection expressions. + let mut projection_indices = + collect_column_indices_in_proj_exprs(projection.expr()); + projection_indices.sort_by_key(|i| *i); + + // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. + // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of hash_join schema fields. + if projection_indices.is_empty() + || (projection_indices.len() == projection_indices.last().unwrap() + 1 + && projection_indices.len() == hash_join.schema().fields().len()) + { + return Ok(Transformed::no(self)); + } + + let new_hash_join = + Arc::new(hash_join.with_projection(Some(projection_indices.clone()))?) + as Arc; + + let builtin_projection_exprs = projection_indices + .iter() + .zip(new_hash_join.schema().fields()) + .map(|(index, field)| { + ( + Arc::new(Column::new(field.name(), *index)) as _, + field.name().to_owned(), + ) + }) + .collect::>(); + + let mut new_projection_exprs = Vec::with_capacity(projection.expr().len()); + for (expr, alias) in projection.expr() { + // Update column index for projection expression since the input schema has been changed: + let Some(expr) = + update_expr_with_projection(expr, &builtin_projection_exprs, false)? + else { + return Ok(Transformed::no(self)); + }; + new_projection_exprs.push((expr, alias.clone())); + } + // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, + // but our projection_exprs in hash join just contain column, so we need to create the new projection + // to keep the original projection. + let new_projection = + ProjectionExec::try_new(new_projection_exprs, new_hash_join.clone())?; + + let children_nodes = self.children_nodes.swap_remove(0).children_nodes; + if is_projection_removable(&new_projection) { + let required_columns = collect_columns_in_plan_schema(&new_hash_join); + Ok(Transformed::yes(Self { + plan: new_hash_join, + required_columns, + schema_mapping: IndexMap::new(), + children_nodes, + })) + } else { + let new_join_node = Self { + plan: new_hash_join, + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes, + }; + Ok(Transformed::no(Self { + plan: Arc::new(new_projection), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes: vec![new_join_node], + })) + } + } + + /// Swaps the projection with its input plan. Input plan does not need any index change or rewrite since + /// it does not change the schema nor refer any column from its input. Only plan and node updates are done. + fn try_swap_trivial(mut self) -> Result> { + // The plan must be a projection. + let Some(projection) = self.plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(self)); + }; + // If the projection does not narrow the schema or it does some calculations, + // we should not try to push it down to have less computation during execution. + // (Making any computation dominates the existance of column elimination) + if projection.expr().len() >= projection.input().schema().fields().len() + || !projection.expr().iter().all(|(expr, _)| { + expr.as_any().is::() || expr.as_any().is::() + }) + { + return Ok(Transformed::no(self)); + } + + let trivial_node = self.children_nodes.swap_remove(0); + let new_projection_nodes = trivial_node + .children_nodes + .into_iter() + .map(|gc| { + Ok(ProjectionOptimizer { + plan: self.plan.clone().with_new_children(vec![gc.plan.clone()])?, + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes: vec![gc], + }) + }) + .collect::>>()?; + + self.plan = trivial_node.plan.with_new_children( + new_projection_nodes + .iter() + .map(|p| p.plan.clone()) + .collect(), + )?; + self.children_nodes = new_projection_nodes; + + // Move the requirements without change. + let child_reqs = mem::take(&mut self.required_columns); + self.children_nodes + .iter_mut() + .for_each(|c| c.required_columns.clone_from(&child_reqs)); + + Ok(Transformed::yes(self)) + } + + /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. + fn try_projected_csv(self) -> Result { + // These plans are known. + let Some(projection) = self.plan.as_any().downcast_ref::() else { + return Ok(self); + }; + let Some(csv) = projection.input().as_any().downcast_ref::() else { + return Ok(self); + }; + + let mut file_scan = csv.base_config().clone(); + // 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 let Some(projection_columns) = + try_collect_alias_free_columns(projection.expr()) + { + let file_projection = file_scan + .projection + .unwrap_or((0..csv.schema().fields().len()).collect()); + // Update a source provider's projected columns according to the given projection expressions. + let new_projections = projection_columns + .iter() + .map(|col| file_projection[col.index()]) + .collect::>(); + file_scan.projection = Some(new_projections); + + Ok(ProjectionOptimizer { + plan: Arc::new(CsvExec::new( + file_scan, + csv.has_header(), + csv.delimiter(), + csv.quote(), + csv.escape(), + csv.file_compression_type, + )), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), // Sources cannot have a mapping. + children_nodes: vec![], + }) + } else { + let required_columns = + collect_column_indices_in_proj_exprs(projection.expr()); + let file_scan_projection = file_scan + .projection + .unwrap_or((0..csv.schema().fields().len()).collect()); + let (used_indices, unused_indices) = + partition_column_indices(file_scan_projection, required_columns); + let new_exprs = update_proj_exprs(projection.expr(), unused_indices)?; + file_scan.projection = Some(used_indices); + let new_csv = ProjectionOptimizer { + plan: Arc::new(CsvExec::new( + file_scan, + csv.has_header(), + csv.delimiter(), + csv.quote(), + csv.escape(), + csv.file_compression_type, + )), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), // Sources cannot have a mapping. + children_nodes: vec![], + }; + + Ok(Self { + plan: Arc::new(ProjectionExec::try_new(new_exprs, new_csv.plan.clone())?) + as Arc, + required_columns: self.required_columns, + schema_mapping: self.schema_mapping, + children_nodes: vec![new_csv], + }) + } + } + + /// 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(self) -> Result { + let plan = self.plan.clone(); + + if is_plan_schema_agnostic(&plan) { + self.try_insert_below_schema_agnostic() + } else if is_plan_requirement_extender(&plan) { + self.try_insert_below_req_extender() + } else { + self.try_insert_below_custom_plans() + } + } + + fn try_insert_below_custom_plans(mut self) -> Result { + let plan = self.plan.clone(); + // Concatenates schemas and do not change requirements. + if let Some(cj) = plan.as_any().downcast_ref::() { + self.try_insert_below_cross_join(cj) + } + // ------------------------------------------------------------------------ + // Joins and aggregations require special attention. + else if let Some(hj) = plan.as_any().downcast_ref::() { + self.try_insert_below_hash_join(hj) + } else if let Some(nlj) = plan.as_any().downcast_ref::() { + self.try_insert_below_nested_loop_join(nlj) + } else if let Some(smj) = plan.as_any().downcast_ref::() { + self.try_insert_below_sort_merge_join(smj) + } else if let Some(shj) = plan.as_any().downcast_ref::() { + self.try_insert_below_symmetric_hash_join(shj) + } else if let Some(agg) = plan.as_any().downcast_ref::() { + if !is_agg_expr_rewritable(agg.aggr_expr()) { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + self.try_insert_below_aggregate(agg) + } else if plan.as_any().is::() + || plan.as_any().is::() + { + let window_exprs = collect_window_expressions(&self.plan); + match self.try_insert_below_window_execs(window_exprs)? { + optimized if optimized.transformed => Ok(optimized.data), + mut no_change => { + no_change.data.children_nodes[0].required_columns = + collect_columns_in_plan_schema( + &no_change.data.children_nodes[0].plan, + ); + Ok(no_change.data) + } + } + } else { + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan) + }); + Ok(self) + } + } + + fn try_insert_below_schema_agnostic(mut self) -> Result { + let plan = self.plan.clone(); + let requirement_map = analyze_requirements(&self); + if all_columns_required(&requirement_map) { + let required_columns = mem::take(&mut self.required_columns); + for c in self.children_nodes.iter_mut() { + c.required_columns.clone_from(&required_columns); + } + } else { + let (new_children, schema_mapping) = + self.insert_multi_projection_below_union(requirement_map)?; + let new_plan = plan.clone().with_new_children( + new_children.iter().map(|c| c.plan.clone()).collect(), + )?; + + self = ProjectionOptimizer { + plan: new_plan, + required_columns: IndexSet::new(), // clear the requirements + schema_mapping, + children_nodes: new_children, + } + } + Ok(self) + } + + fn try_insert_below_req_extender(mut self) -> Result { + let Some(columns) = self.plan.expressions() else { + return Ok(self); + }; + self.required_columns + .extend(columns.into_iter().flat_map(|e| collect_columns(&e))); + let requirement_map = analyze_requirements(&self); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let plan = self.plan.clone(); + let (mut new_child, schema_mapping) = + self.insert_projection(requirement_map)?; + + let Some(new_plan) = + plan.update_expressions(&ExprMapping::new(schema_mapping.clone()))? + else { + return internal_err!( + "Plans implementing expressions() must also implement update_expressions()" + ); + }; + + self = ProjectionOptimizer { + plan: new_plan, + required_columns: IndexSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child.swap_remove(0)], + } + } + Ok(self) + } + + /// Attempts to insert projection nodes below a `CrossJoinExec` node in the query execution plan. + /// + /// This method first analyzes the requirements for both the left and right sides of the cross join. Depending on these + /// requirements, it may insert projections on either or both sides of the join. Specifically, if not all columns are + /// required on either side, it inserts the necessary projection nodes to streamline the join operation. If all columns + /// are required from both sides, it updates the required columns accordingly without adding any projections. This + /// optimization is crucial for reducing the computational overhead in cross join operations. + 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) = analyze_requirements_of_joins( + cj.left(), + cj.right(), + &self.required_columns, + left_size, + ); + 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_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(), + )); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + // Left child needs a projection. + (false, true) => { + let required_columns = mem::take(&mut self.required_columns); + let mut right_child = self.children_nodes.swap_remove(1); + right_child.required_columns = + update_right_child_requirements(&required_columns, left_size); + + let (new_left_child, mut left_schema_mapping) = + self.insert_projection_below_left_child(analyzed_join_left)?; + let plan = Arc::new(CrossJoinExec::new( + new_left_child.plan.clone(), + right_child.plan.clone(), + )) as _; + + let new_left_size = new_left_child.plan.schema().fields().len(); + left_schema_mapping = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, + ); + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + // Right child needs a projection. + (true, false) => { + let required_columns = mem::take(&mut self.required_columns); + let mut left_child = self.children_nodes.swap_remove(0); + left_child.required_columns = + collect_left_used_columns(required_columns, left_size); + + let (new_right_child, mut right_schema_mapping) = + self.insert_projection_below_right_child(analyzed_join_right)?; + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + let plan = Arc::new(CrossJoinExec::new( + left_child.plan.clone(), + new_right_child.plan.clone(), + )); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = IndexSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + Ok(self) + } + + /// Attempts to insert a projection below a HashJoinExec node in a query plan. + /// + /// This function modifies the projection optimizer by analyzing and potentially + /// inserting new projections below the HashJoinExec node based on the join conditions + /// and the required columns in the query. The process involves analyzing both left + /// and right sides of the join, updating equivalence and non-equivalence conditions, + /// and reorganizing the required columns as needed. This function supports various + /// join types, including Inner, Left, Right, Full, LeftAnti, LeftSemi, RightAnti, + /// and RightSemi joins. + fn try_insert_below_hash_join( + mut self, + hj: &HashJoinExec, + ) -> Result { + match hj.join_type() { + JoinType::Inner + | JoinType::Left + | JoinType::Right + | JoinType::Full + | JoinType::LeftSemi + | JoinType::LeftAnti => { + let join_left_input_size = hj.left().schema().fields().len(); + let join_projection = hj + .projection + .clone() + .unwrap_or((0..hj.schema().fields().len()).collect()); + + let hj_left_requirements = collect_hj_left_requirements( + &self.required_columns, + &join_projection, + join_left_input_size, + hj.left().schema(), + hj.on(), + hj.filter(), + ); + + let hj_right_requirements = collect_hj_right_requirements( + &self.required_columns, + &join_projection, + join_left_input_size, + hj.right().schema(), + hj.on(), + hj.filter(), + ); + + let left_input_columns = collect_columns_in_plan_schema(hj.left()); + let keep_left_same = left_input_columns.iter().all(|left_input_column| { + hj_left_requirements.contains(left_input_column) + }); + + let right_input_columns = collect_columns_in_plan_schema(hj.right()); + let keep_right_same = + right_input_columns.iter().all(|right_input_column| { + hj_right_requirements.contains(right_input_column) + }); + + match (keep_left_same, keep_right_same) { + (false, false) => { + let (new_left_node, new_right_node) = update_hj_children( + &hj_left_requirements, + &hj_right_requirements, + self.children_nodes, + hj, + )?; + + let (left_mapping, right_mapping) = update_hj_children_mapping( + &hj_left_requirements, + &hj_right_requirements, + ); + + let new_on = + update_join_on(hj.on(), &left_mapping, &right_mapping); + let new_filter = + rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); + + let new_projection = update_hj_projection( + hj.projection.clone(), + hj.left().schema(), + hj_left_requirements, + left_mapping, + right_mapping, + join_left_input_size, + ); + + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + new_right_node.plan.clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes: vec![new_left_node, new_right_node], + }) + } + (false, true) => { + let (new_left_node, right_node) = update_hj_left_child( + &hj_left_requirements, + &hj_right_requirements, + self.children_nodes, + hj, + )?; + + let (left_mapping, right_mapping) = update_hj_children_mapping( + &hj_left_requirements, + &hj_right_requirements, + ); + + let new_on = + update_join_on(hj.on(), &left_mapping, &right_mapping); + let new_filter = + rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); + let new_projection = update_hj_projection( + hj.projection.clone(), + hj.left().schema(), + hj_left_requirements, + left_mapping, + right_mapping, + join_left_input_size, + ); + + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + right_node.plan.clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes: vec![new_left_node, right_node], + }) + } + (true, false) => { + let (left_node, new_right_node) = update_hj_right_child( + &hj_left_requirements, + &hj_right_requirements, + self.children_nodes, + hj, + )?; + + let (left_mapping, right_mapping) = update_hj_children_mapping( + &hj_left_requirements, + &hj_right_requirements, + ); + + let new_on = + update_join_on(hj.on(), &left_mapping, &right_mapping); + let new_filter = + rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); + let new_projection = update_hj_projection( + hj.projection.clone(), + hj.left().schema(), + hj_left_requirements, + left_mapping, + right_mapping, + join_left_input_size, + ); + + let new_hash_join = HashJoinExec::try_new( + left_node.plan.clone(), + new_right_node.plan.clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes: vec![left_node, new_right_node], + }) + } + (true, true) => { + self.required_columns = IndexSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }); + Ok(self) + } + } + } + JoinType::RightSemi | JoinType::RightAnti => { + let join_projection = hj + .projection + .clone() + .unwrap_or((0..hj.right().schema().fields().len()).collect()); + + let hj_left_requirements = collect_right_hj_left_requirements( + hj.left().schema(), + hj.on(), + hj.filter(), + ); + + let hj_right_requirements = collect_right_hj_right_requirements( + &self.required_columns, + &join_projection, + hj.right().schema(), + hj.on(), + hj.filter(), + ); + + let left_input_columns = collect_columns_in_plan_schema(hj.left()); + let keep_left_same = left_input_columns.iter().all(|left_input_column| { + hj_left_requirements.contains(left_input_column) + }); + + let right_input_columns = collect_columns_in_plan_schema(hj.right()); + let keep_right_same = + right_input_columns.iter().all(|right_input_column| { + hj_right_requirements.contains(right_input_column) + }); + + match (keep_left_same, keep_right_same) { + (false, false) => { + let (new_left_node, new_right_node) = update_hj_children( + &hj_left_requirements, + &hj_right_requirements, + self.children_nodes, + hj, + )?; + + let (left_mapping, right_mapping) = update_hj_children_mapping( + &hj_left_requirements, + &hj_right_requirements, + ); + + let new_on = + update_join_on(hj.on(), &left_mapping, &right_mapping); + let new_filter = + rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); + + let new_projection = update_hj_projection_right( + hj.projection.clone(), + right_mapping, + ); + + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + new_right_node.plan.clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes: vec![new_left_node, new_right_node], + }) + } + (false, true) => { + let (new_left_node, right_node) = update_hj_left_child( + &hj_left_requirements, + &hj_right_requirements, + self.children_nodes, + hj, + )?; + + let (left_mapping, right_mapping) = update_hj_children_mapping( + &hj_left_requirements, + &hj_right_requirements, + ); + + let new_on = + update_join_on(hj.on(), &left_mapping, &right_mapping); + let new_filter = + rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); + let new_projection = update_hj_projection_right( + hj.projection.clone(), + left_mapping, + ); + + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + right_node.plan.clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes: vec![new_left_node, right_node], + }) + } + (true, false) => { + let (left_node, new_right_node) = update_hj_right_child( + &hj_left_requirements, + &hj_right_requirements, + self.children_nodes, + hj, + )?; + + let (left_mapping, right_mapping) = update_hj_children_mapping( + &hj_left_requirements, + &hj_right_requirements, + ); + + let new_on = + update_join_on(hj.on(), &left_mapping, &right_mapping); + let new_filter = + rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); + let new_projection = update_hj_projection_right( + hj.projection.clone(), + right_mapping, + ); + + let new_hash_join = HashJoinExec::try_new( + left_node.plan.clone(), + new_right_node.plan.clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), + children_nodes: vec![left_node, new_right_node], + }) + } + (true, true) => { + self.required_columns = IndexSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }); + Ok(self) + } + } + } + } + } + + /// Attempts to insert a projection below a NestedLoopJoinExec node in a query plan. + /// + /// This function modifies the projection optimizer by analyzing and potentially + /// inserting new projections below the NestedLoopJoinExec node based on the join conditions + /// and the required columns in the query. The process involves analyzing both left + /// and right sides of the join, updating equivalence and non-equivalence conditions, + /// and reorganizing the required columns as needed. This function supports various + /// join types, including Inner, Left, Right, Full, LeftAnti, LeftSemi, RightAnti, + /// and RightSemi joins. + 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. + match nlj.join_type() { + JoinType::RightAnti | JoinType::RightSemi => { + self.required_columns = + update_right_requirements(self.required_columns, left_size); + } + _ => {} + } + 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) = analyze_requirements_of_joins( + nlj.left(), + nlj.right(), + &self.required_columns, + 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(), + )?); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let required_columns = mem::take(&mut self.required_columns); + let mut right_child = self.children_nodes.swap_remove(1); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &analyzed_join_right, + &IndexMap::new(), + ); + let (new_left_child, mut left_schema_mapping) = + self.insert_projection_below_left_child(analyzed_join_left)?; + + right_child.required_columns = + update_right_child_requirements(&required_columns, left_size); + + let plan = Arc::new(NestedLoopJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?); + let new_left_size = new_left_child.plan.schema().fields().len(); + left_schema_mapping = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, + ); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let required_columns = mem::take(&mut self.required_columns); + let mut left_child = self.children_nodes.swap_remove(0); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &IndexMap::new(), + &analyzed_join_right, + ); + let (new_right_child, mut right_schema_mapping) = self + .insert_projection_below_right_child(analyzed_join_right)?; + + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + left_child.required_columns = + collect_left_used_columns(required_columns, left_size); + + let plan = Arc::new(NestedLoopJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = IndexSet::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, + &IndexMap::new(), + ); + let (new_left_child, left_schema_mapping) = + self.insert_projection_below_left_child(analyzed_join_left)?; + + let plan = Arc::new(NestedLoopJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?); + + 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: IndexSet::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(), + &IndexMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_right_child(analyzed_join_right)?; + + let plan = Arc::new(NestedLoopJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?); + + 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: IndexSet::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) + } + + /// Attempts to insert a projection below a SortMergeJoinExec node in a query plan. + /// + /// This function modifies the projection optimizer by analyzing and potentially + /// inserting new projections below the SortMergeJoinExec node based on the join conditions + /// and the required columns in the query. The process involves analyzing both left + /// and right sides of the join, updating equivalence and non-equivalence conditions, + /// and reorganizing the required columns as needed. This function supports various + /// join types, including Inner, Left, Right, Full, LeftAnti, LeftSemi, RightAnti, + /// and RightSemi joins. + 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. + match smj.join_type() { + JoinType::RightAnti | JoinType::RightSemi => { + self.required_columns = + update_right_requirements(self.required_columns, left_size); + } + _ => {} + } + 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) = analyze_requirements_of_joins( + smj.left(), + smj.right(), + &self.required_columns, + 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, + )?); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let required_columns = mem::take(&mut self.required_columns); + let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + smj.on(), + &analyzed_join_left, + &IndexMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &analyzed_join_right, + &IndexMap::new(), + ); + let (new_left_child, mut left_schema_mapping) = + self.insert_projection_below_left_child(analyzed_join_left)?; + right_child.required_columns = required_columns + .iter() + .filter(|col| col.index() >= left_size) + .map(|col| Column::new(col.name(), col.index() - left_size)) + .collect(); + 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, + )?); + let new_left_size = new_left_child.plan.schema().fields().len(); + left_schema_mapping = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, + ); + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let required_columns = mem::take(&mut self.required_columns); + let mut left_child = self.children_nodes.swap_remove(0); + let new_on = update_equivalence_conditions( + smj.on(), + &IndexMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &IndexMap::new(), + &analyzed_join_right, + ); + let (new_right_child, mut right_schema_mapping) = self + .insert_projection_below_right_child(analyzed_join_right)?; + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + left_child.required_columns = + collect_left_used_columns(required_columns, left_size); + + 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, + )?); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = IndexSet::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, + &IndexMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &analyzed_join_left, + &IndexMap::new(), + ); + let (new_left_child, left_schema_mapping) = + self.insert_projection_below_left_child(analyzed_join_left)?; + + 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, + )?); + + 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: IndexSet::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(), + &IndexMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &IndexMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_right_child(analyzed_join_right)?; + + 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, + )?); + + 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: IndexSet::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) + } + + /// Attempts to insert a projection below a SymmetricHashJoinExec node in a query plan. + /// + /// This function modifies the projection optimizer by analyzing and potentially + /// inserting new projections below the SymmetricHashJoinExec node based on the join conditions + /// and the required columns in the query. The process involves analyzing both left + /// and right sides of the join, updating equivalence and non-equivalence conditions, + /// and reorganizing the required columns as needed. This function supports various + /// join types, including Inner, Left, Right, Full, LeftAnti, LeftSemi, RightAnti, + /// and RightSemi joins. + 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. + match shj.join_type() { + JoinType::RightAnti | JoinType::RightSemi => { + self.required_columns = + update_right_requirements(self.required_columns, left_size); + } + _ => {} + } + 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) = analyze_requirements_of_joins( + shj.left(), + shj.right(), + &self.required_columns, + 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 new_left_sort_exprs = + shj.left_sort_exprs().map(|sort_expr| { + update_sort_expressions(sort_expr, &schema_mapping) + }); + let new_right_sort_exprs = + shj.left_sort_exprs().map(|sort_expr| { + update_sort_expressions(sort_expr, &schema_mapping) + }); + + 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(), + new_left_sort_exprs, + new_right_sort_exprs, + shj.partition_mode(), + )?); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let required_columns = mem::take(&mut self.required_columns); + let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + shj.on(), + &analyzed_join_left, + &IndexMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &analyzed_join_right, + &IndexMap::new(), + ); + let (new_left_child, mut left_schema_mapping) = + self.insert_projection_below_left_child(analyzed_join_left)?; + + right_child.required_columns = + update_right_child_requirements(&required_columns, left_size); + + 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(), + )?); + let new_left_size = new_left_child.plan.schema().fields().len(); + left_schema_mapping = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, + ); + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let required_columns = mem::take(&mut self.required_columns); + let mut left_child = self.children_nodes.swap_remove(0); + let new_on = update_equivalence_conditions( + shj.on(), + &IndexMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &IndexMap::new(), + &analyzed_join_right, + ); + let (new_right_child, mut right_schema_mapping) = self + .insert_projection_below_right_child(analyzed_join_right)?; + + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + left_child.required_columns = + collect_left_used_columns(required_columns, left_size); + 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(), + )?); + + self = ProjectionOptimizer { + plan, + required_columns: IndexSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = IndexSet::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, + &IndexMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &analyzed_join_left, + &IndexMap::new(), + ); + let (new_left_child, left_schema_mapping) = + self.insert_projection_below_left_child(analyzed_join_left)?; + + 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(), + )?); + + 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: IndexSet::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(), + &IndexMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &IndexMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_right_child(analyzed_join_right)?; + + 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(), + )?); + + 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: IndexSet::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_expr_indices = agg + .aggr_expr() + .iter() + .enumerate() + .filter(|(idx, _expr)| !required_indices.contains(&(idx + group_columns_len))) + .map(|(idx, _expr)| idx) + .collect::>(); + + if !unused_aggr_expr_indices.is_empty() { + let new_plan = AggregateExec::try_new( + *agg.mode(), + agg.group_expr().clone(), + agg.aggr_expr() + .iter() + .enumerate() + .filter(|(idx, _expr)| !unused_aggr_expr_indices.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() + .flat_map(|(e, _alias)| collect_columns(e)) + .collect(); + self.children_nodes[0].required_columns.extend( + new_plan.aggr_expr().iter().flat_map(|e| { + e.expressions() + .iter() + .flat_map(collect_columns) + .collect::>() + }), + ); + self.plan = Arc::new(new_plan); + self.required_columns = IndexSet::new(); + } else { + 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::>(); + let aggr_columns = match agg.mode() { + AggregateMode::Final | AggregateMode::FinalPartitioned => { + let mut group_expr_len = agg.group_expr().expr().iter().count(); + agg.aggr_expr() + .iter() + .flat_map(|e| { + e.state_fields().map(|field| { + field + .iter() + .map(|field| { + group_expr_len += 1; + Column::new(field.name(), group_expr_len - 1) + }) + .collect::>() + }) + }) + .flatten() + .collect::>() + } + _ => agg + .aggr_expr() + .iter() + .flat_map(|e| { + e.expressions() + .iter() + .flat_map(collect_columns) + .collect::>() + }) + .collect::>(), + }; + self.children_nodes[0].required_columns.extend( + aggr_columns + .into_iter() + .chain(group_columns) + .chain(filter_columns), + ) + } + Ok(self) + } + + fn try_insert_below_window_execs( + mut self, + window_exprs: Vec>, + ) -> Result> { + let Some(columns) = self.plan.expressions() else { + return Ok(Transformed::no(self)); + }; + self.required_columns + .extend(columns.into_iter().flat_map(|e| collect_columns(&e))); + + let requirement_map = analyze_requirements(&self); + if !all_columns_required(&requirement_map) { + if window_exec_required( + self.plan.children()[0].schema().fields().len(), + &requirement_map, + ) { + if window_exprs.iter().any(|expr| { + expr.clone() + .update_expression(&ExprMapping::new(self.schema_mapping.clone())) + .is_none() + }) { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .filter(|col| { + col.index() + < self.plan.schema().fields().len() - window_exprs.len() + }) + .cloned() + .collect(); + return Ok(Transformed::no(self)); + } + let plan = self.plan.clone(); + let (new_child, schema_mapping, window_usage) = + self.insert_projection_below_window(requirement_map)?; + + let mut with_removed_exprs = schema_mapping + .iter() + .map(|(col1, col2)| { + ( + Arc::new(col1.clone()) as Arc, + Some(Arc::new(col2.clone()) as Arc), + ) + }) + .collect::>(); + window_usage.iter().for_each(|(column, usage)| { + if !*usage { + with_removed_exprs.iter_mut().for_each(|(p1, p2)| { + if p1 + .as_any() + .downcast_ref::() + .map(|c| c == column) + .unwrap_or(false) + { + *p2 = None + } + }); + } + }); + let Some(new_plan) = plan + .clone() + .update_expressions(&ExprMapping::new(schema_mapping.clone()))? + else { + return internal_err!( + "Plans implementing expressions() must also implement update_expressions()" + ); + }; + let required_columns = collect_columns_in_plan_schema(&plan); + self = ProjectionOptimizer { + plan: new_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 = + mem::take(&mut self.required_columns) + .into_iter() + .filter(|col| { + col.index() + < self.plan.schema().fields().len() - window_exprs.len() + }) + .collect(); + } + Ok(Transformed::no(self)) + } + + /// 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<(Vec, IndexMap)> { + // 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 (used_columns, unused_columns) = + partition_column_requirements(requirement_map); + + let projected_exprs = convert_projection_exprs(used_columns); + 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 new_requirements = inserted_projections + .iter() + .map(|inserted_projection| { + collect_columns_in_plan_schema(inserted_projection) + }) + .collect::>(); + + let new_mapping = + calculate_column_mapping(&self.required_columns, &unused_columns); + + 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: IndexMap::new(), + children_nodes: vec![child], + }) + .collect(); + + Ok((inserted_projection_nodes, new_mapping)) + } + + /// Multi-child version of `insert_projection` for `UnionExec`'s. + fn insert_multi_projection_below_union( + self, + requirement_map: ColumnRequirements, + ) -> Result<(Vec, IndexMap)> { + // 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 (used_columns, unused_columns) = + partition_column_requirements(requirement_map); + + let projected_exprs = convert_projection_exprs(used_columns); + 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 new_requirements = inserted_projections + .iter() + .map(|inserted_projection| { + collect_columns_in_plan_schema(inserted_projection) + }) + .collect::>(); + let new_mapping = + calculate_column_mapping(&self.required_columns, &unused_columns); + + 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: IndexMap::new(), + children_nodes: vec![child], + }) + .collect(); + Ok((inserted_projection_nodes, 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, IndexMap)> { + let initial_right_child = self.children_nodes[1].plan.clone(); + + let (left_used_columns, left_unused_columns) = + partition_column_requirements(requirement_map_left); + let left_schema_mapping = + calculate_column_mapping(&left_used_columns, &left_unused_columns); + let (right_used_columns, right_unused_columns) = + partition_column_requirements(requirement_map_right); + let right_schema_mapping = + calculate_column_mapping(&right_used_columns, &right_unused_columns); + + let (new_left_child, new_right_child) = self + .insert_multi_projections_below_join_inner( + left_used_columns, + right_used_columns, + )?; + + let new_left_size = new_left_child.plan.schema().fields().len(); + let right_projection = collect_columns_in_plan_schema(&new_right_child.plan); + let initial_right_schema = initial_right_child.schema(); + let maintained_columns = initial_right_schema + .fields() + .iter() + .enumerate() + .filter(|(idx, field)| { + right_projection.contains(&Column::new(field.name(), *idx)) + }) + .collect::>(); + + let mut all_mappings = left_schema_mapping; + for (idx, field) in maintained_columns { + all_mappings.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() { + all_mappings.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, all_mappings)) + } + + fn insert_multi_projections_below_join_inner( + mut self, + left_used_columns: IndexSet, + right_used_columns: IndexSet, + ) -> Result<(Self, Self)> { + let child_plan = self.plan.children().remove(0); + let projected_exprs = convert_projection_exprs(left_used_columns); + let inserted_projection = + Arc::new(ProjectionExec::try_new(projected_exprs, child_plan)?) as _; + + let required_columns = collect_columns_in_plan_schema(&inserted_projection); + let left_inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: IndexMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(0)], + }; + + let child_plan = self.plan.children().remove(1); + let projected_exprs = convert_projection_exprs(right_used_columns); + let inserted_projection = + Arc::new(ProjectionExec::try_new(projected_exprs, child_plan)?) as _; + + let required_columns = collect_columns_in_plan_schema(&inserted_projection); + let right_inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: IndexMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(0)], + }; + Ok((left_inserted_projection, right_inserted_projection)) + } + + /// Left child version of `insert_projection` for joins. + fn insert_projection_below_left_child( + mut self, + requirement_map_left: ColumnRequirements, + ) -> Result<(Self, IndexMap)> { + let child_plan = self.plan.children().remove(0); + let (used_columns, unused_columns) = + partition_column_requirements(requirement_map_left); + + let new_mapping = calculate_column_mapping(&used_columns, &unused_columns); + let projected_exprs = convert_projection_exprs(used_columns); + let inserted_projection = + Arc::new(ProjectionExec::try_new(projected_exprs, child_plan)?) as _; + + let required_columns = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: IndexMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(0)], + }; + + Ok((inserted_projection, new_mapping)) + } + + /// Right child version of `insert_projection` for joins. + fn insert_projection_below_right_child( + mut self, + requirement_map_right: ColumnRequirements, + ) -> Result<(Self, IndexMap)> { + let child_plan = self.plan.children().remove(1); + let (used_columns, unused_columns) = + partition_column_requirements(requirement_map_right); + + let new_mapping = calculate_column_mapping(&used_columns, &unused_columns); + let projected_exprs = convert_projection_exprs(used_columns); + let inserted_projection = + Arc::new(ProjectionExec::try_new(projected_exprs, child_plan)?) as _; + + let required_columns = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: IndexMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(0)], + }; + Ok((inserted_projection, new_mapping)) + } + + /// `insert_projection` for windows. + fn insert_projection_below_window( + self, + requirement_map: ColumnRequirements, + ) -> Result<(Self, IndexMap, ColumnRequirements)> { + let original_schema_len = self.plan.schema().fields().len(); + let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map + .into_iter() + .partition(|(column, _used)| column.index() < original_schema_len); + let (used_columns, mut unused_columns) = partition_column_requirements(base); + let projected_exprs = convert_projection_exprs(used_columns); + + for (col, used) in window.iter() { + if !used { + unused_columns.insert(col.clone()); + } + } + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs, + self.plan.children()[0].clone(), + )?) as _; + + let new_mapping = + calculate_column_mapping(&self.required_columns, &unused_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: IndexMap::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 after + /// the optimizations in its input subtree, which leads to re-update the mapping after removal. + fn index_updater(mut self: ProjectionOptimizer) -> Result> { + let mut all_mappings = self + .children_nodes + .iter_mut() + .map(|node| mem::take(&mut node.schema_mapping)) + .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(), + )?; + update_mapping(&mut self, 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(), + )?; + update_mapping(&mut self, 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(), + )?; + update_mapping(&mut self, 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(), + )?; + update_mapping(&mut self, 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(), + )?; + update_mapping(&mut self, 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(), + )?; + update_mapping(&mut self, all_mappings) + } + // ------------------------------------------------------------------------ + else if let Some(filter) = plan_any.downcast_ref::() { + self.plan = rewrite_filter( + filter.predicate(), + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + update_mapping(&mut self, 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], + )?; + update_mapping(&mut self, all_mappings) + } else if let Some(sort) = plan_any.downcast_ref::() { + self.plan = rewrite_sort( + sort, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + update_mapping(&mut self, all_mappings) + } else if let Some(sortp_merge) = + plan_any.downcast_ref::() + { + self.plan = rewrite_sort_preserving_merge( + sortp_merge, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + update_mapping(&mut self, 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(_cj) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + update_mapping_cross(&mut self, all_mappings) + } else if let Some(hj) = plan_any.downcast_ref::() { + self = self.update_hash_join(hj, all_mappings)?; + } else if let Some(nlj) = plan_any.downcast_ref::() { + self = self.update_nested_loop_join(nlj, all_mappings)?; + } else if let Some(smj) = plan_any.downcast_ref::() { + self = self.update_sort_merge_join(smj, all_mappings)?; + } else if let Some(shj) = plan_any.downcast_ref::() { + self = self.update_symmetric_hash_join(shj, all_mappings)?; + } else if let Some(agg) = plan_any.downcast_ref::() { + if agg.aggr_expr().iter().any(|expr| { + expr.clone() + .update_expression(&ExprMapping::new(all_mappings[0].clone())) + .is_none() + && !self.children_nodes[0].schema_mapping.is_empty() + }) { + self = preserve_requirements(self)?; + return Ok(Transformed::no(self)); + } + 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)); + }; + } 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)); + }; + update_mapping(&mut self, 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)); + }; + update_mapping(&mut self, 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 existing_columns.iter().enumerate() { + 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: IndexSet::new(), + schema_mapping: IndexMap::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 { + return Ok(Transformed::no(self)); + } + } else { + let res = self.plan.clone().with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + + self.plan = res; + } + + Ok(Transformed::yes(self)) + } + + /// 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. + let Some(projection_columns) = try_collect_alias_free_columns(projection.expr()) + else { + return Ok(self); + }; + + // 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 child_col_names = child_columns + .iter() + .map(|col| col.name().to_string()) + .collect::>(); + if child_columns + .iter() + .all(|child_col| projection_columns.contains(child_col)) + && child_col_names.len() == child_columns.len() + { + // 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_columns + .iter() + .enumerate() + .filter_map(|(idx, col)| { + if col.index() != idx { + Some((Column::new(col.name(), idx), col.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() + } + } + + Ok(self) + } + + fn update_hash_join( + mut self, + hj: &HashJoinExec, + mut all_mappings: Vec>, + ) -> Result { + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let projection = hj + .projection + .clone() + .unwrap_or((0..hj.schema().fields().len()).collect()); + let new_on = update_join_on(hj.on(), &left_mapping, &right_mapping); + 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: left_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: right_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::Right, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + match hj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + let index_mapping = left_mapping + .iter() + .map(|(col1, col2)| (col1.index(), col2.index())) + .chain(right_mapping.iter().map(|(col1, col2)| { + ( + col1.index() + + self.children_nodes[0].plan.schema().fields().len(), + col2.index() + + self.children_nodes[0].plan.schema().fields().len(), + ) + })) + .collect::>(); + let new_projection = projection + .into_iter() + .map(|idx| *index_mapping.get(&idx).unwrap_or(&idx)) + .collect::>(); + let some_projection = new_projection + .first() + .map(|first| *first != 0) + .unwrap_or(true) + || !new_projection.windows(2).all(|w| w[0] + 1 == w[1]); + self.plan = HashJoinExec::try_new( + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + new_on, + new_filter, + hj.join_type(), + if some_projection { + Some(new_projection) + } else { + None + }, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .map(|plan| Arc::new(plan) as _)?; + } + JoinType::LeftSemi | JoinType::LeftAnti => { + let index_mapping = left_mapping + .iter() + .map(|(col1, col2)| (col1.index(), col2.index())) + .collect::>(); + let new_projection = projection + .into_iter() + .map(|idx| *index_mapping.get(&idx).unwrap_or(&idx)) + .collect::>(); + let some_projection = new_projection + .first() + .map(|first| *first != 0) + .unwrap_or(true) + || !new_projection.windows(2).all(|w| w[0] + 1 == w[1]) + || self.children_nodes[0].plan.schema().fields().len() + != new_projection.len(); + self.plan = HashJoinExec::try_new( + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + new_on, + new_filter, + hj.join_type(), + if some_projection { + Some(new_projection) + } else { + None + }, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .map(|plan| Arc::new(plan) as _)?; + self.schema_mapping = left_mapping; + } + JoinType::RightSemi | JoinType::RightAnti => { + let index_mapping = right_mapping + .iter() + .map(|(col1, col2)| (col1.index(), col2.index())) + .collect::>(); + + let mut new_projection = projection + .into_iter() + .map(|idx| *index_mapping.get(&idx).unwrap_or(&idx)) + .collect::>(); + new_projection.sort_by_key(|ind| *ind); + let some_projection = new_projection + .first() + .map(|first| *first != 0) + .unwrap_or(true) + || !new_projection.windows(2).all(|w| w[0] + 1 == w[1]) + || self.children_nodes[1].plan.schema().fields().len() + != new_projection.len(); + + self.plan = HashJoinExec::try_new( + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + new_on, + new_filter, + hj.join_type(), + if some_projection { + Some(new_projection) + } else { + None + }, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .map(|plan| Arc::new(plan) as _)?; + self.schema_mapping = right_mapping; + } + } + Ok(self) + } + + fn update_nested_loop_join( + mut self, + nlj: &NestedLoopJoinExec, + mut all_mappings: Vec>, + ) -> Result { + 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())) + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), + 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(), + &left_mapping, + &right_mapping, + left_size, + )?; + match nlj.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[0].extend(new_right); + } + JoinType::LeftSemi | JoinType::LeftAnti => all_mappings.push(left_mapping), + JoinType::RightAnti | JoinType::RightSemi => all_mappings.push(right_mapping), + }; + update_mapping(&mut self, all_mappings); + Ok(self) + } + + fn update_sort_merge_join( + mut self, + smj: &SortMergeJoinExec, + mut all_mappings: Vec>, + ) -> Result { + 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())) + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), + 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(), + &left_mapping, + &right_mapping, + left_size, + )?; + match smj.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[0].extend(new_right); + } + JoinType::LeftSemi | JoinType::LeftAnti => all_mappings.push(left_mapping), + JoinType::RightAnti | JoinType::RightSemi => all_mappings.push(right_mapping), + }; + update_mapping(&mut self, all_mappings); + Ok(self) + } + + fn update_symmetric_hash_join( + mut self, + shj: &SymmetricHashJoinExec, + mut all_mappings: Vec>, + ) -> Result { + 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())) + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), + 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(), + &left_mapping, + &right_mapping, + left_size, + )?; + match shj.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[0].extend(new_right); + } + JoinType::LeftSemi | JoinType::LeftAnti => all_mappings.push(left_mapping), + JoinType::RightAnti | JoinType::RightSemi => all_mappings.push(right_mapping), + }; + update_mapping(&mut self, all_mappings); + Ok(self) + } +} + +impl ConcreteTreeNode for ProjectionOptimizer { + fn children(&self) -> Vec<&Self> { + self.children_nodes.iter().collect_vec() + } + + fn take_children(mut self) -> (Self, Vec) { + let children = mem::take(&mut self.children_nodes); + (self, children) + } + + fn with_new_children(mut self, children: Vec) -> Result { + self.children_nodes = children; + + self = match self.index_updater()? { + new_node if new_node.transformed => new_node.data, + same_node => ProjectionOptimizer::new_default(same_node.data.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 {} + } +} + +impl PhysicalOptimizerRule for OptimizeProjections { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + let final_schema_determinant = find_final_schema_determinant(&plan); + // Collect initial columns requirements from the plan's schema. + let initial_requirements = + collect_columns_in_plan_schema(&final_schema_determinant); + + let mut optimizer = ProjectionOptimizer::new_default(final_schema_determinant); + + // Insert the initial requirements to the root node, and run the rule. + optimizer.required_columns.clone_from(&initial_requirements); + let mut optimized = optimizer.transform_down(|o: ProjectionOptimizer| { + o.adjust_node_with_requirements().map(Transformed::yes) + })?; + // When some projections are removed after the rule, we know that all columns of + // the initial schema still exist, but their order may be changed. Ensure the final + // optimized plan satisfies the initial schema order. + optimized = optimized + .map_data(|node| satisfy_initial_schema(node, initial_requirements))?; + let new_child = optimized.data.plan; + + if is_plan_schema_determinant(&plan) { + Ok(new_child) + } else { + update_children(plan, new_child) + } + } + + fn name(&self) -> &str { + "OptimizeProjections" + } + + fn schema_check(&self) -> bool { + true + } +} + +/// If the schema of the plan can be different than its input schema, +/// then the function returns true; otherwise, false. +fn is_plan_schema_determinant(plan: &Arc) -> bool { + let plan_any = plan.as_any(); + + plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan.children().len() != 1 +} + +/// Given a plan, the function returns the closest node to the root which updates the schema. +fn find_final_schema_determinant( + plan: &Arc, +) -> Arc { + if is_plan_schema_determinant(plan) { + plan.clone() + } else { + plan.children() + .first() + .map(find_final_schema_determinant) + .unwrap_or(plan.clone()) + } +} + +/// Given a plan and a child plan, the function updates the child of the node +/// which is the closest node to the root and modifing the schema. +fn update_children( + plan: Arc, + new_child: Arc, +) -> Result> { + let children = plan.children(); + let Some(child) = children.first() else { + return Ok(plan.clone()); + }; + + if is_plan_schema_determinant(child) { + plan.with_new_children(vec![new_child]) + } else { + let new_child = plan + .children() + .first() + .map(|c| update_children(c.clone(), new_child)) + .transpose() + .map(|new_plan| new_plan.unwrap_or(plan.clone()))?; + plan.with_new_children(vec![new_child]) + } +} + +/// Filters the expressions of a [`ProjectionExec`] according to the given used column indices. +fn collect_used_columns( + projection_exprs: &[(Arc, String)], + used_columns: &IndexSet, +) -> Vec<(Arc, String)> { + let used_indices = used_columns + .iter() + .map(|column| column.index()) + .collect::>(); + projection_exprs + .iter() + .enumerate() + .filter_map(|(idx, (expr, alias))| { + if used_indices.contains(&idx) { + Some((expr.clone(), alias.clone())) + } else { + None + } + }) + .collect::>() +} + +fn collect_left_used_columns( + required_columns: IndexSet, + left_size: usize, +) -> IndexSet { + required_columns + .into_iter() + .filter(|col| col.index() < left_size) + .collect() +} + +/// Collects all fields of a schema from a given execution plan and converts them into a [`IndexSet`] of [`Column`]. +fn collect_columns_in_plan_schema(plan: &Arc) -> IndexSet { + plan.schema() + .fields() + .iter() + .enumerate() + .map(|(i, f)| Column::new(f.name(), i)) + .collect() +} + +/// Collects all columns involved in the join's equivalence and non-equivalence conditions, +/// adjusting the indices for columns from the right table by adding the size of the left table. +fn collect_columns_in_join_conditions( + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, + left_size: usize, + join_left_schema: SchemaRef, + join_right_schema: SchemaRef, +) -> IndexSet { + 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 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) + .collect::>() +} + +/// Given expressions of a projection, the function collects all mentioned columns into a vector. +fn collect_column_indices_in_proj_exprs( + exprs: &[(Arc, String)], +) -> Vec { + exprs + .iter() + .flat_map(|(expr, _)| collect_columns(expr)) + .map(|col| col.index()) + .collect::>() + .into_iter() + .collect() +} + +/// Collects the columns that the projection requires from its input. +fn collect_projection_input_requirements( + required_columns: IndexSet, + projection_exprs: &[(Arc, String)], +) -> IndexSet { + required_columns + .into_iter() + .filter_map(|c| projection_exprs.get(c.index()).map(|e| e.0.clone())) + .flat_map(|e| collect_columns(&e)) + .collect::>() +} + +fn collect_hj_left_requirements( + all_requirements: &IndexSet, + join_projection: &[usize], + join_left_input_size: usize, + join_left_schema: SchemaRef, + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, +) -> IndexSet { + let mut hj_left_requirements = all_requirements + .iter() + .filter_map(|req| { + if join_projection[req.index()] < join_left_input_size { + Some(Column::new(req.name(), join_projection[req.index()])) + } else { + None + } + }) + .collect::>(); + hj_left_requirements.extend( + on.iter() + .flat_map(|(left_on, _)| collect_columns(left_on)) + .collect::>(), + ); + hj_left_requirements.extend( + filter + .map(|filter| { + filter + .column_indices() + .iter() + .filter_map(|col_ind| { + if col_ind.side == JoinSide::Left { + Some(Column::new( + join_left_schema.fields()[col_ind.index].name(), + col_ind.index, + )) + } else { + None + } + }) + .collect::>() + }) + .unwrap_or_default(), + ); + hj_left_requirements +} + +fn collect_right_hj_left_requirements( + join_left_schema: SchemaRef, + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, +) -> IndexSet { + let mut hj_left_requirements = IndexSet::new(); + hj_left_requirements.extend( + on.iter() + .flat_map(|(left_on, _)| collect_columns(left_on)) + .collect::>(), + ); + hj_left_requirements.extend( + filter + .map(|filter| { + filter + .column_indices() + .iter() + .filter_map(|col_ind| { + if col_ind.side == JoinSide::Left { + Some(Column::new( + join_left_schema.fields()[col_ind.index].name(), + col_ind.index, + )) + } else { + None + } + }) + .collect::>() + }) + .unwrap_or_default(), + ); + hj_left_requirements +} + +fn collect_hj_right_requirements( + all_requirements: &IndexSet, + join_projection: &[usize], + join_left_input_size: usize, + join_right_schema: SchemaRef, + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, +) -> IndexSet { + let mut hj_right_requirements = all_requirements + .iter() + .filter_map(|req| { + if join_projection[req.index()] >= join_left_input_size { + Some(Column::new( + req.name(), + join_projection[req.index()] - join_left_input_size, + )) + } else { + None + } + }) + .collect::>(); + hj_right_requirements.extend( + on.iter() + .flat_map(|(_, right_on)| collect_columns(right_on)) + .collect::>(), + ); + + hj_right_requirements.extend( + filter + .map(|filter| { + filter + .column_indices() + .iter() + .filter_map(|col_ind| { + if col_ind.side == JoinSide::Right { + Some(Column::new( + join_right_schema.fields()[col_ind.index].name(), + col_ind.index, + )) + } else { + None + } + }) + .collect::>() + }) + .unwrap_or_default(), + ); + hj_right_requirements +} + +fn collect_right_hj_right_requirements( + all_requirements: &IndexSet, + join_projection: &[usize], + join_right_schema: SchemaRef, + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, +) -> IndexSet { + let mut hj_right_requirements = all_requirements + .iter() + .map(|req| Column::new(req.name(), join_projection[req.index()])) + .collect::>(); + hj_right_requirements.extend( + on.iter() + .flat_map(|(_, right_on)| collect_columns(right_on)) + .collect::>(), + ); + + hj_right_requirements.extend( + filter + .map(|filter| { + filter + .column_indices() + .iter() + .filter_map(|col_ind| { + if col_ind.side == JoinSide::Right { + Some(Column::new( + join_right_schema.fields()[col_ind.index].name(), + col_ind.index, + )) + } else { + None + } + }) + .collect::>() + }) + .unwrap_or_default(), + ); + hj_right_requirements +} + +fn collect_window_expressions( + window_exec: &Arc, +) -> Vec> { + if let Some(window) = window_exec.as_any().downcast_ref::() { + window.window_expr().to_vec() + } else if let Some(bounded_window) = + window_exec.as_any().downcast_ref::() + { + bounded_window.window_expr().to_vec() + } else { + vec![] + } +} + +/// Given the expressions of a projection, checks if the projection causes +/// any renaming or constructs a non-`Column` physical expression. If all +/// expressions are `Column`, then they are collected and returned. If not, +/// the function returns `None`. +fn try_collect_alias_free_columns( + exprs: &[(Arc, String)], +) -> Option> { + let mut columns = vec![]; + for (expr, alias) in exprs { + match expr.as_any().downcast_ref::() { + Some(column) if column.name() == alias => columns.push(column.clone()), + _ => return None, + } + } + Some(columns) +} + +#[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 projecting 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_with_projection( + expr: &Arc, + projected_exprs: &[(Arc, String)], + sync_with_child: bool, +) -> Result>> { + let mut state = RewriteState::Unchanged; + let new_expr = expr.clone().transform_up(|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.eq(projected_column).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.data)) +} + +/// Rewrites the sort expressions with new index values. +fn update_sort_expressions( + sort_exprs: &[PhysicalSortExpr], + mapping: &IndexMap, +) -> LexOrdering { + let expr_map = ExprMapping::new(mapping.clone()); + sort_exprs + .iter() + .filter_map(|sort_expr| { + expr_map + .update_expression(sort_expr.expr.clone()) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }) + }) + .collect::>() +} + +/// Rewrites the window expressions with new index values. +fn update_window_exprs( + window_exprs: &[Arc], + mapping: &IndexMap, +) -> Option>> { + window_exprs + .iter() + .map(|window_expr| { + window_expr + .clone() + .update_expression(&ExprMapping::new(mapping.clone())) + }) + .collect::>>() +} + +/// Rewrites the aggregate expressions with new index values. +fn update_aggregate_exprs( + aggregate_exprs: &[Arc], + mapping: &IndexMap, +) -> Option>> { + aggregate_exprs + .iter() + .map(|aggr_expr| { + aggr_expr + .clone() + .update_expression(&ExprMapping::new(mapping.clone())) + }) + .collect::>>() +} + +/// Rewrites the expressions in equivalence condition of a join with new index values. +fn update_join_on( + join_on: JoinOnRef, + left_mapping: &IndexMap, + right_mapping: &IndexMap, +) -> JoinOn { + let left_expr_map = ExprMapping::new(left_mapping.clone()); + let right_expr_map = ExprMapping::new(right_mapping.clone()); + join_on + .iter() + .filter_map(|(left, right)| { + match ( + left_expr_map.update_expression(left.clone()), + right_expr_map.update_expression(right.clone()), + ) { + (Some(left), Some(right)) => Some((left, right)), + _ => None, + } + }) + .collect() +} + +/// Updates the equivalence conditions of the joins according to the new indices of columns. +fn update_equivalence_conditions( + on: &[(PhysicalExprRef, PhysicalExprRef)], + 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(|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() + .data, + right_col + .clone() + .transform_up(|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() + .data, + ) + }) + .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(), + ) + }) +} + +fn update_right_child_requirements( + required_columns: &IndexSet, + left_size: usize, +) -> IndexSet { + required_columns + .iter() + .filter(|col| col.index() >= left_size) + .map(|col| Column::new(col.name(), col.index() - left_size)) + .collect() +} + +fn update_mapping( + node: &mut ProjectionOptimizer, + mut child_mappings: Vec>, +) { + if node.schema_mapping.is_empty() { + node.schema_mapping = child_mappings.swap_remove(0); + } else { + let child_map = child_mappings.swap_remove(0); + node.schema_mapping = node + .schema_mapping + .iter() + .map(|(initial, new)| { + ( + initial.clone(), + child_map.get(new).cloned().unwrap_or(new.clone()), + ) + }) + .collect() + } +} + +fn update_mapping_cross( + node: &mut ProjectionOptimizer, + mut child_mappings: Vec>, +) { + if node.schema_mapping.is_empty() { + node.schema_mapping = child_mappings.swap_remove(0); + node.schema_mapping + .extend(child_mappings[0].iter().map(|(initial, new)| { + ( + Column::new( + initial.name(), + initial.index() + + node.children_nodes[0].plan.schema().fields().len(), + ), + Column::new( + new.name(), + new.index() + node.children_nodes[0].plan.schema().fields().len(), + ), + ) + })); + } else { + let mut self_node_map = node.schema_mapping.clone(); + node.schema_mapping = child_mappings.swap_remove(0); + node.schema_mapping + .extend(child_mappings[0].iter().map(|(initial, new)| { + ( + Column::new( + initial.name(), + initial.index() + + node.children_nodes[0].plan.schema().fields().len(), + ), + Column::new( + new.name(), + new.index() + node.children_nodes[0].plan.schema().fields().len(), + ), + ) + })); + + node.schema_mapping = node + .schema_mapping + .clone() + .into_iter() + .map(|(initial, new)| { + if let Some((match_i, _match_f)) = + self_node_map.clone().iter().find(|(_i, f)| initial == **f) + { + self_node_map.swap_remove(match_i); + (match_i.clone(), new) + } else { + (initial, new) + } + }) + .collect(); + node.schema_mapping.extend(self_node_map); + } +} + +fn update_right_mapping( + right_schema_mapping: IndexMap, + left_size: usize, +) -> IndexMap { + 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() +} + +fn update_right_requirements( + required_columns: IndexSet, + left_size: usize, +) -> IndexSet { + required_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect() +} + +/// After removal of unused columns in the input table, the columns in projection expression +/// need an update. This function calculates the new positions and updates the indices of columns. +fn update_proj_exprs( + exprs: &[(Arc, String)], + unused_indices: Vec, +) -> Result, String)>> { + exprs + .iter() + .map(|(expr, alias)| { + let new_expr = expr + .clone() + .transform_up(|expr: Arc| { + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::no(expr)); + }; + let diff = unused_indices + .iter() + .filter(|idx| **idx < column.index()) + .count(); + Ok(Transformed::yes(Arc::new(Column::new( + column.name(), + column.index() - diff, + )))) + })? + .data; + Ok((new_expr, alias.to_owned())) + }) + .collect::>>() +} + +fn update_hj_children( + hj_left_requirements: &IndexSet, + hj_right_requirements: &IndexSet, + mut children: Vec, + hj: &HashJoinExec, +) -> Result<(ProjectionOptimizer, ProjectionOptimizer)> { + let mut ordered_hj_left_requirements = + hj_left_requirements.iter().cloned().collect_vec(); + ordered_hj_left_requirements.sort_by_key(|col| col.index()); + let left_projection_exprs = ordered_hj_left_requirements + .into_iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req) as Arc, name) + }) + .collect::>(); + let new_left_projection = + ProjectionExec::try_new(left_projection_exprs, hj.left().clone())?; + let new_left_projection_arc = Arc::new(new_left_projection) as Arc; + let new_left_requirements = collect_columns_in_plan_schema(&new_left_projection_arc); + let new_left_node = ProjectionOptimizer { + plan: new_left_projection_arc, + required_columns: new_left_requirements, + schema_mapping: IndexMap::new(), + children_nodes: vec![children.swap_remove(0)], + }; + + let mut ordered_hj_right_requirements = + hj_right_requirements.iter().cloned().collect_vec(); + ordered_hj_right_requirements.sort_by_key(|col| col.index()); + let right_projection_exprs = ordered_hj_right_requirements + .into_iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req) as Arc, name) + }) + .collect::>(); + + let new_right_projection = + ProjectionExec::try_new(right_projection_exprs, hj.right().clone())?; + let new_right_projection_arc = + Arc::new(new_right_projection) as Arc; + let new_right_requirements = + collect_columns_in_plan_schema(&new_right_projection_arc); + let new_right_node = ProjectionOptimizer { + plan: new_right_projection_arc, + required_columns: new_right_requirements, + schema_mapping: IndexMap::new(), + children_nodes: vec![children.swap_remove(0)], + }; + + Ok((new_left_node, new_right_node)) +} + +fn update_hj_left_child( + hj_left_requirements: &IndexSet, + hj_right_requirements: &IndexSet, + mut children: Vec, + hj: &HashJoinExec, +) -> Result<(ProjectionOptimizer, ProjectionOptimizer)> { + let mut ordered_hj_left_requirements = + hj_left_requirements.iter().cloned().collect_vec(); + ordered_hj_left_requirements.sort_by_key(|col| col.index()); + let left_projection_exprs = ordered_hj_left_requirements + .into_iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req) as Arc, name) + }) + .collect::>(); + let new_left_projection = + ProjectionExec::try_new(left_projection_exprs, hj.left().clone())?; + let new_left_projection_arc = Arc::new(new_left_projection) as Arc; + let new_left_requirements = collect_columns_in_plan_schema(&new_left_projection_arc); + let new_left_node = ProjectionOptimizer { + plan: new_left_projection_arc, + required_columns: new_left_requirements, + schema_mapping: IndexMap::new(), + children_nodes: vec![children.swap_remove(0)], + }; + + let mut right_node = children.swap_remove(0); + right_node + .required_columns + .clone_from(hj_right_requirements); + + Ok((new_left_node, right_node)) +} + +fn update_hj_right_child( + hj_left_requirements: &IndexSet, + hj_right_requirements: &IndexSet, + mut children: Vec, + hj: &HashJoinExec, +) -> Result<(ProjectionOptimizer, ProjectionOptimizer)> { + let mut ordered_hj_right_requirements = + hj_right_requirements.iter().cloned().collect_vec(); + ordered_hj_right_requirements.sort_by_key(|col| col.index()); + let right_projection_exprs = ordered_hj_right_requirements + .into_iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req) as Arc, name) + }) + .collect::>(); + let new_right_projection = + ProjectionExec::try_new(right_projection_exprs, hj.right().clone())?; + let new_right_projection_arc = + Arc::new(new_right_projection) as Arc; + let new_right_requirements = + collect_columns_in_plan_schema(&new_right_projection_arc); + let new_right_node = ProjectionOptimizer { + plan: new_right_projection_arc, + required_columns: new_right_requirements, + schema_mapping: IndexMap::new(), + children_nodes: vec![children.swap_remove(1)], + }; + + let mut left_node = children.swap_remove(0); + left_node.required_columns.clone_from(hj_left_requirements); + + Ok((left_node, new_right_node)) +} + +fn update_hj_children_mapping( + hj_left_requirements: &IndexSet, + hj_right_requirements: &IndexSet, +) -> (IndexMap, IndexMap) { + let mut left_mapping = hj_left_requirements.iter().cloned().collect_vec(); + left_mapping.sort_by_key(|col| col.index()); + let left_mapping = left_mapping + .into_iter() + .enumerate() + .map(|(idx, col)| (col.clone(), Column::new(col.name(), idx))) + .collect::>(); + let mut right_mapping = hj_right_requirements.iter().collect_vec(); + right_mapping.sort_by_key(|col| col.index()); + let right_mapping = right_mapping + .into_iter() + .enumerate() + .map(|(idx, col)| (col.clone(), Column::new(col.name(), idx))) + .collect::>(); + (left_mapping, right_mapping) +} + +fn update_hj_projection( + projection: Option>, + hj_left_schema: SchemaRef, + hj_left_requirements: IndexSet, + left_mapping: IndexMap, + right_mapping: IndexMap, + join_left_input_size: usize, +) -> Option> { + projection.map(|projection| { + projection + .iter() + .map(|ind| { + if ind < &hj_left_schema.fields().len() { + left_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .map(|(_, target)| target.index()) + .unwrap_or(*ind) + } else { + right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == *ind - join_left_input_size + }) + .map(|(_, target)| target.index()) + .unwrap_or(*ind) + + hj_left_requirements.len() + } + }) + .collect() + }) +} + +fn update_hj_projection_right( + projection: Option>, + right_mapping: IndexMap, +) -> Option> { + projection.map(|projection| { + projection + .iter() + .map(|ind| { + right_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .map(|(_, target)| target.index()) + .unwrap_or(*ind) + }) + .collect() + }) +} + +/// Rewrites a filter execution plan with updated column indices. +fn rewrite_filter( + predicate: &Arc, + input_plan: Arc, + mapping: &IndexMap, +) -> Result> { + let map = ExprMapping::new(mapping.clone()); + let Some(new_expr) = map.update_expression(predicate.clone()) else { + return internal_err!("Filter predicate cannot be rewritten"); + }; + FilterExec::try_new(new_expr, input_plan).map(|plan| Arc::new(plan) as _) +} + +fn rewrite_hj_filter( + filter: Option<&JoinFilter>, + left_mapping: &IndexMap, + right_mapping: &IndexMap, +) -> Option { + filter.map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_ind| { + if col_ind.side == JoinSide::Left { + ColumnIndex { + index: left_mapping + .iter() + .find(|(initial, _)| initial.index() == col_ind.index) + .map(|(_, target)| target.index()) + .unwrap_or(col_ind.index), + side: JoinSide::Left, + } + } else { + ColumnIndex { + index: right_mapping + .iter() + .find(|(initial, _)| initial.index() == col_ind.index) + .map(|(_, target)| target.index()) + .unwrap_or(col_ind.index), + side: JoinSide::Right, + } + } + }) + .collect(), + filter.schema().clone(), + ) + }) +} + +/// Rewrites a projection execution plan with updated column indices. +fn rewrite_projection( + projection: &ProjectionExec, + input_plan: Arc, + mapping: &IndexMap, +) -> Result> { + let mapping = ExprMapping::new(mapping.clone()); + ProjectionExec::try_new( + projection + .expr() + .iter() + .filter_map(|(expr, alias)| { + mapping + .update_expression(expr.clone()) + .map(|e| (e, alias.clone())) + }) + .collect::>(), + input_plan, + ) + .map(|plan| Arc::new(plan) as _) +} + +/// Rewrites a repartition execution plan with updated column indices. +fn rewrite_repartition( + partitioning: &Partitioning, + input_plan: Arc, + mapping: &IndexMap, +) -> Result> { + let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { + let mapping = ExprMapping::new(mapping.clone()); + let new_exprs = exprs + .iter() + .filter_map(|e| mapping.update_expression(e.clone())) + .collect(); + Partitioning::Hash(new_exprs, *size) + } else { + partitioning.clone() + }; + RepartitionExec::try_new(input_plan, new_partitioning).map(|plan| Arc::new(plan) as _) +} + +/// Rewrites a sort execution plan with updated column indices. +fn rewrite_sort( + sort: &SortExec, + input_plan: Arc, + mapping: &IndexMap, +) -> Result> { + let new_sort_exprs = update_sort_expressions(sort.expr(), mapping); + Ok(Arc::new( + SortExec::new(new_sort_exprs, input_plan) + .with_fetch(sort.fetch()) + .with_preserve_partitioning(sort.preserve_partitioning()), + ) as _) +} + +/// Rewrites a sort preserving merge execution plan with updated column indices. +fn rewrite_sort_preserving_merge( + sort: &SortPreservingMergeExec, + input_plan: Arc, + mapping: &IndexMap, +) -> Result> { + let new_sort_exprs = update_sort_expressions(sort.expr(), mapping); + Ok(Arc::new( + SortPreservingMergeExec::new(new_sort_exprs, input_plan).with_fetch(sort.fetch()), + ) as _) +} + +fn rewrite_nested_loop_join( + nlj: &NestedLoopJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + left_mapping: &IndexMap, + right_mapping: &IndexMap, + 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: left_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: right_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 _) +} + +/// Rewrites a sort merge join execution plan. This function modifies a SortMergeJoinExec plan with +/// new left and right input plans, updating join conditions and filters according to the provided mappings. +fn rewrite_sort_merge_join( + smj: &SortMergeJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + left_mapping: &IndexMap, + right_mapping: &IndexMap, + left_size: usize, +) -> Result> { + let new_on = update_join_on(smj.on(), left_mapping, right_mapping); + 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: left_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: right_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 _) +} + +/// Rewrites a symmetric hash join execution plan. Adjusts a SymmetricHashJoinExec plan with +/// new input plans and column mappings, maintaining the original join logic but with updated references. +fn rewrite_symmetric_hash_join( + shj: &SymmetricHashJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + left_mapping: &IndexMap, + right_mapping: &IndexMap, + left_size: usize, +) -> Result> { + let new_on = update_join_on(shj.on(), left_mapping, right_mapping); + 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: left_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: right_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(), + ) + }); + + let left_expr_map = ExprMapping::new(left_mapping.clone()); + let right_expr_map = ExprMapping::new(right_mapping.clone()); + let new_left_sort_exprs = shj.left_sort_exprs().map(|exprs| { + exprs + .iter() + .filter_map(|sort_expr| { + left_expr_map + .update_expression(sort_expr.expr.clone()) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }) + }) + .collect() + }); + let new_right_sort_exprs = shj.left_sort_exprs().map(|exprs| { + exprs + .iter() + .filter_map(|sort_expr| { + right_expr_map + .update_expression(sort_expr.expr.clone()) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }) + }) + .collect() + }); + SymmetricHashJoinExec::try_new( + left_input_plan, + right_input_plan, + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + new_left_sort_exprs, + new_right_sort_exprs, + shj.partition_mode(), + ) + .map(|plan| Arc::new(plan) as _) +} + +/// Rewrites an aggregate execution plan. This function updates an AggregateExec plan using a new +/// input plan and column mappings. It adjusts group-by expressions, aggregate expressions, and filters. +fn rewrite_aggregate( + agg: &AggregateExec, + input_plan: Arc, + mapping: &IndexMap, +) -> Result>> { + let expr_map = ExprMapping::new(mapping.clone()); + let new_group_by = PhysicalGroupBy::new( + agg.group_expr() + .expr() + .iter() + .filter_map(|(expr, alias)| { + expr_map + .update_expression(expr.clone()) + .map(|e| (e, alias.to_string())) + }) + .collect(), + agg.group_expr() + .null_expr() + .iter() + .filter_map(|(expr, alias)| { + expr_map + .update_expression(expr.clone()) + .map(|e| (e, alias.to_string())) + }) + .collect(), + agg.group_expr().groups().to_vec(), + ); + let new_agg_expr = + if let Some(new_agg_expr) = update_aggregate_exprs(agg.aggr_expr(), mapping) { + new_agg_expr + } else { + return Ok(None); + }; + let mapping = ExprMapping::new(mapping.clone()); + let new_filter = agg + .filter_expr() + .iter() + .filter_map(|opt_expr| { + opt_expr.clone().map(|expr| mapping.update_expression(expr)) + }) + .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.with_limit(agg.limit())) as _)) +} + +/// Rewrites a window aggregate execution plan. Modifies a WindowAggExec by updating its input plan and expressions +/// based on the provided column mappings, ensuring the window functions are correctly applied to the new plan structure. +fn rewrite_window_aggregate( + w_agg: &WindowAggExec, + input_plan: Arc, + mapping: &IndexMap, +) -> Result>> { + let new_window = + if let Some(new_window) = update_window_exprs(w_agg.window_expr(), mapping) { + new_window + } else { + return Ok(None); + }; + let mapping = ExprMapping::new(mapping.clone()); + let new_partition_keys = w_agg + .partition_keys + .iter() + .filter_map(|k| mapping.update_expression(k.clone())) + .collect(); + WindowAggExec::try_new(new_window, input_plan, new_partition_keys) + .map(|plan| Some(Arc::new(plan) as _)) +} + +/// Rewrites a bounded window aggregate execution plan. Updates a BoundedWindowAggExec plan with a new input plan +/// and modified window expressions according to provided column mappings, maintaining the logic of bounded window functions. +fn rewrite_bounded_window_aggregate( + bw_agg: &BoundedWindowAggExec, + input_plan: Arc, + mapping: &IndexMap, +) -> Result>> { + let new_window = + if let Some(new_window) = update_window_exprs(bw_agg.window_expr(), mapping) { + new_window + } else { + return Ok(None); + }; + let mapping = ExprMapping::new(mapping.clone()); + let new_partition_keys = bw_agg + .partition_keys + .iter() + .filter_map(|k| mapping.update_expression(k.clone())) + .collect(); + BoundedWindowAggExec::try_new( + new_window, + input_plan, + new_partition_keys, + bw_agg.input_order_mode.clone(), + ) + .map(|plan| Some(Arc::new(plan) as _)) +} + +/// If the plan does not change the input schema and does not refer any +/// input column, the function returns true. These kind of plans can swap +/// the order with projections without any further adaptation. +fn is_plan_schema_agnostic(plan: &Arc) -> bool { + let plan_any = plan.as_any(); + plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() +} + +fn is_plan_requirement_extender(plan: &Arc) -> bool { + let plan_any = plan.as_any(); + plan_any.is::() + || plan_any.is::() + || plan_any.is::() + || plan_any.is::() +} + +/// 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().is::() || expr.as_any().is::() +} + +/// Compare the inputs and outputs of the projection. All expressions must be +/// columns without alias, and projection does not change the order of fields. +/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, +/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. +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() +} + +/// Tries to rewrite the [`AggregateExpr`] with the existing expressions to keep on optimization. +fn is_agg_expr_rewritable(aggr_expr: &[Arc<(dyn AggregateExpr)>]) -> bool { + aggr_expr.iter().all(|expr| expr.expressions().is_empty()) +} + +/// 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(node: &ProjectionOptimizer) -> ColumnRequirements { + let mut requirement_map = ColumnRequirements::new(); + let columns_in_schema = collect_columns_in_plan_schema(&node.plan); + columns_in_schema.into_iter().for_each(|col| { + let contains = node.required_columns.contains(&col); + requirement_map.insert(col, contains); + }); + requirement_map +} + +/// Analyzes the column requirements for join operations between left and right children plans. +/// +/// This function compares the required columns from the left and right children with the existing columns in their +/// respective schemas. It determines if there are any redundant fields and creates a mapping to indicate whether +/// each column is required. The function returns a pair of `ColumnRequirements`, one for each child. +/// +/// The caller must ensure that the join node extends its requirements if the node's plan can introduce new columns. +/// +/// # Arguments +/// * `left_child`: Reference to the execution plan of the left child. +/// * `right_child`: Reference to the execution plan of the right child. +/// * `required_columns`: Set of columns that are required by the parent plan. +/// * `left_size`: Size of the left child's schema, used to adjust the index of right child's columns. +/// +/// # Returns +/// A tuple containing two `ColumnRequirements`: +/// - The first element represents the column requirements for the left child. +/// - The second element represents the column requirements for the right child. +fn analyze_requirements_of_joins( + left_child: &Arc, + right_child: &Arc, + required_columns: &IndexSet, + left_size: usize, +) -> (ColumnRequirements, ColumnRequirements) { + let columns_in_schema = collect_columns_in_plan_schema(left_child) + .into_iter() + .chain( + collect_columns_in_plan_schema(right_child) + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)), + ); + + let requirement_map = columns_in_schema + .into_iter() + .map(|col| { + if 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) +} + +/// 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) +} + +/// Checks if all columns in the input schema are required by the projection. +fn all_input_columns_required( + input_columns: &IndexSet, + projection_requires: &IndexSet, +) -> bool { + input_columns + .iter() + .all(|input_column| projection_requires.contains(input_column)) +} + +/// Given a `ColumnRequirements`, it partitions the required and redundant columns. +fn partition_column_requirements( + requirements: ColumnRequirements, +) -> (IndexSet, IndexSet) { + let mut required = IndexSet::new(); + let mut unused = IndexSet::new(); + for (col, is_req) in requirements { + if is_req { + required.insert(col); + } else { + unused.insert(col); + } + } + (required, unused) +} + +fn partition_column_indices( + file_scan_projection: Vec, + required_columns: Vec, +) -> (Vec, Vec) { + let used_indices = file_scan_projection + .iter() + .enumerate() + .filter_map(|(idx, csv_indx)| { + if required_columns.contains(&idx) { + Some(*csv_indx) + } else { + None + } + }) + .collect::>(); + let unused_indices = file_scan_projection + .into_iter() + .enumerate() + .filter_map(|(idx, csv_idx)| { + if used_indices.contains(&csv_idx) { + None + } else { + Some(idx) + } + }) + .collect::>(); + (used_indices, unused_indices) +} + +// 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, +) -> Result { + let mut column_ref_map: IndexMap = IndexMap::new(); + // Collect the column references' usage in the parent projection. + projection.expr().iter().try_for_each(|(expr, _)| { + expr.apply(|expr| { + Ok({ + if let Some(column) = expr.as_any().downcast_ref::() { + *column_ref_map.entry(column.clone()).or_default() += 1; + } + TreeNodeRecursion::Continue + }) + })?; + Ok(()) as Result<()> + })?; + Ok(column_ref_map.iter().any(|(column, count)| { + *count > 1 && !is_expr_trivial(&child_projection.expr()[column.index()].0) + })) +} + +/// 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( + mut po: ProjectionOptimizer, + initial_requirements: IndexSet, +) -> Result { + if po.schema_mapping.is_empty() { + // The initial schema is already satisfied, no further action required. + Ok(po) + } else { + let mut initial_requirements_ordered = + initial_requirements.into_iter().collect_vec(); + initial_requirements_ordered.sort_by_key(|expr| expr.index()); + let projected_exprs = initial_requirements_ordered + .into_iter() + .map(|col| { + // If there is a change, get the new index. + let final_column = po.schema_mapping.swap_remove(&col).unwrap_or(col); + let final_column_name = final_column.name().to_string(); + let new_col = Arc::new(final_column) as Arc; + (new_col, final_column_name) + }) + .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: IndexSet::new(), + schema_mapping: IndexMap::new(), // Reset schema mapping as we've now satisfied the initial schema + children_nodes: vec![po], // Keep the original node as the child + }) + } +} + +fn preserve_requirements(po: ProjectionOptimizer) -> Result { + if po.schema_mapping.is_empty() { + // 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 current_fields = collect_columns_in_plan_schema(&po.children_nodes[0].plan); + let sorted_current_fields = current_fields + .into_iter() + .sorted_by_key(|f| f.index()) + .collect::>(); + let mut projected_exprs = vec![]; + for (idx, field) in po.children_nodes[0] + .plan + .schema() + .fields() + .iter() + .enumerate() + { + let column = Column::new(field.name(), idx); + let target = sorted_current_fields[po + .schema_mapping + .get(&column) + .map(|col| col.index()) + .unwrap_or(idx)] + .clone(); + projected_exprs.push(target); + } + let projected_exprs = projected_exprs + .into_iter() + .map(|expr| (Arc::new(expr.clone()) as _, expr.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: po.required_columns.clone(), + schema_mapping: IndexMap::new(), // Reset schema mapping as we've now satisfied the initial schema + children_nodes: vec![po], // Keep the original node as the child + }) + } +} + +fn window_exec_required( + original_schema_len: usize, + requirements: &ColumnRequirements, +) -> bool { + requirements + .iter() + .filter(|(column, _used)| column.index() >= original_schema_len) + .any(|(_column, used)| *used) +} + +/// When a field in a schema is decided to be redundant and planned to be dropped +/// since it is not required from the plans above, some of the other fields will +/// potentially move to the left side by one. That will change the plans above +/// referring to that field, and they need to update their expressions. This function +/// calculates those index changes and records old and new column expressions in a map. +fn calculate_column_mapping( + required_columns: &IndexSet, + unused_columns: &IndexSet, +) -> IndexMap { + let mut new_mapping = IndexMap::new(); + for col in 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), + ); + } + } + new_mapping +} + +/// Given a set of column expression, constructs a vector having the tuples of `PhysicalExpr` +/// and string alias to be used in creation of `ProjectionExec`. Aliases are the name of columns. +fn convert_projection_exprs( + columns: IndexSet, +) -> Vec<(Arc, String)> { + let mut new_expr = columns.into_iter().collect::>(); + new_expr.sort_by_key(|column| column.index()); + new_expr + .into_iter() + .map(|column| { + let name = column.name().to_string(); + (Arc::new(column) as Arc, name) + }) + .collect() +} + +fn extend_left_mapping_with_right( + mut left_schema_mapping: IndexMap, + right_child_plan: &Arc, + left_size: usize, + new_left_size: usize, +) -> IndexMap { + left_schema_mapping.extend( + right_child_plan + .schema() + .fields() + .iter() + .enumerate() + .map(|(idx, field)| { + ( + Column::new(field.name(), left_size + idx), + Column::new(field.name(), new_left_size + idx), + ) + }) + .collect::>(), + ); + left_schema_mapping +} + +/// Calculates the count of removed (unused) columns that precede a given column index. +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 +} + +/// Maps the indices of required columns in a parent projection node to the corresponding indices in its child. +/// +/// # Example +/// Projection is required to have columns at "@0:a - @1:b - @2:c" +/// +/// Projection does "a@2 as a, b@0 as b, c@1 as c" +/// +/// Then, projection inserts requirements into its child with these updated indices: "@0:b - @1:c - @2:a" +fn map_parent_reqs_to_input_reqs( + requirements: &IndexSet, + projection_columns: &[Column], +) -> IndexSet { + requirements + .iter() + .map(|column| projection_columns[column.index()].clone()) + .collect::>() +} + +/// Calculates the index changes of columns after the removal of a projection. +/// It compares the `columns` with the columns in the projection and, if a change +/// is observed, maps the old index to the new index value in an IndexMap. +fn index_changes_after_projection_removal( + columns: IndexSet, + projection_columns: &[Column], +) -> IndexMap { + 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() +} + +#[cfg(test)] +mod tests { + use super::*; + use std::any::Any; + 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::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::{get_plan_string, ExecutionPlan}; + + use arrow_schema::{DataType, Field, Schema, SortOptions}; + use datafusion_common::config::ConfigOptions; + use datafusion_common::{ + plan_err, JoinSide, JoinType, Result, ScalarValue, Statistics, + }; + use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_expr::{ + ColumnarValue, Operator, ScalarUDF, ScalarUDFImpl, Signature, Volatility, + WindowFrame, + }; + use datafusion_physical_expr::expressions::{ + rank, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, RowNumber, + Sum, + }; + use datafusion_physical_expr::window::BuiltInWindowExpr; + use datafusion_physical_expr::{ + Partitioning, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, + }; + use datafusion_physical_plan::joins::{ + HashJoinExec, PartitionMode, SymmetricHashJoinExec, + }; + use datafusion_physical_plan::union::UnionExec; + use datafusion_physical_plan::InputOrderMode; + + #[derive(Debug)] + struct AddOne { + signature: Signature, + } + + impl AddOne { + fn new() -> Self { + Self { + signature: Signature::uniform( + 1, + vec![DataType::Int32], + Volatility::Immutable, + ), + } + } + } + + impl ScalarUDFImpl for AddOne { + fn as_any(&self) -> &dyn Any { + self + } + fn name(&self) -> &str { + "add_one" + } + fn signature(&self) -> &Signature { + &self.signature + } + fn return_type(&self, args: &[DataType]) -> Result { + if !matches!(args.first(), Some(&DataType::Int32)) { + return plan_err!("add_one only accepts Int32 arguments"); + } + Ok(DataType::Int32) + } + fn invoke(&self, _args: &[ColumnarValue]) -> Result { + unimplemented!() + } + } + + 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(ScalarUDF::new_from_impl(AddOne::new())), + 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, + )), + 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(ScalarUDF::new_from_impl(AddOne::new())), + 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, + )), + 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_with_projection(&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(ScalarUDF::new_from_impl(AddOne::new())), + 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, + )), + 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(ScalarUDF::new_from_impl(AddOne::new())), + 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, + )), + 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_with_projection(&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@0 as new_b, c@1 + e@2 as binary, b@0 as newest_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[b, c, 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 = [ + "CoalescePartitionsExec", + " ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@2 as d]", + " 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_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 = + OptimizeProjections::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_hash_join_after_projection() -> Result<()> { + // SELECT t1.c as c_from_left, t1.b as b_from_left, t1.a as a_from_left, t2.c as c_from_right + // FROM t1 JOIN t2 ON t1.b = t2.c WHERE t1.b - (1 + t2.a) <= t2.a + t1.c + let left_csv = create_simple_csv_exec(); + let right_csv = create_simple_csv_exec(); + + let join: Arc = Arc::new(HashJoinExec::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, + None, + PartitionMode::Auto, + true, + )?); + 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("c", 7)), "c_from_right".to_string()), + ], + join.clone(), + )?); + 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, c@7 as c_from_right]", + " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " 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())?; + + // HashJoinExec only returns result after projection. Because there are some alias columns in the projection, the ProjectionExec is not removed. + let expected = [ + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@3 as c_from_right]", + " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@4]", + " 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 projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (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("c", 7)), "c".to_string()), + ], + join.clone(), + )?); + + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + // Comparing to the previous result, this projection don't have alias columns either change the order of output fields. So the ProjectionExec is removed. + let expected = [ + "HashJoinExec: mode=Auto, 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, projection=[a@0, b@1, c@2, c@4]", + " 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); + + 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], preserve_partitioning=[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]", + " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC], preserve_partitioning=[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_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 = [ + "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], 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], 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], has_header=false" + ]; + + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) + } + + #[test] + + fn test_optimize_projections_filter_sort() -> Result<()> { + 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], preserve_partitioning=[false]", + " 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], preserve_partitioning=[false]", + " ProjectionExec: expr=[c@1 as c, a@0 as x]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, c], has_header=false"]; + + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_optimize_projections_left_anti() -> Result<()> { + let csv_left = create_simple_csv_exec(); + let window = Arc::new(WindowAggExec::try_new( + vec![Arc::new(BuiltInWindowExpr::new( + Arc::new(RowNumber::new("ROW_NUMBER()".to_string(), &DataType::Int32)), + &[], + &[], + Arc::new(WindowFrame::new(None)), + ))], + csv_left, + vec![], + )?); + let projection = Arc::new(ProjectionExec::try_new( + vec![ + ( + Arc::new(Column::new("ROW_NUMBER()", 5)), + "ROW_NUMBER()".to_string(), + ), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ( + Arc::new(Column::new("ROW_NUMBER()", 5)), + "ROW_NUMBER()".to_string(), + ), + ], + window, + )?); + let sort = Arc::new(SortExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("ROW_NUMBER()", 4)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("d", 2)), + options: SortOptions::default(), + }, + ], + projection, + )); + + let csv_right = create_simple_csv_exec(); + let bounded_window = Arc::new(BoundedWindowAggExec::try_new( + vec![Arc::new(BuiltInWindowExpr::new( + Arc::new(rank("RANK()".to_string(), &DataType::Int32)), + &[], + &[], + Arc::new(WindowFrame::new(None)), + ))], + csv_right, + vec![], + InputOrderMode::Linear, + )?); + + let join = Arc::new(HashJoinExec::try_new( + sort, + bounded_window, + vec![(Arc::new(Column::new("a", 1)), Arc::new(Column::new("b", 1)))], + Some(JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("ROW_NUMBER()", 0)), + Operator::Lt, + Arc::new(Column::new("RANK", 1)), + )), + vec![ + ColumnIndex { + index: 4, + side: JoinSide::Left, + }, + ColumnIndex { + index: 5, + side: JoinSide::Right, + }, + ], + Schema::new(vec![ + Field::new("inter_rownumber", DataType::Int32, true), + Field::new("inter_avg_b", DataType::Float32, true), + ]), + )), + &JoinType::LeftAnti, + Some(vec![1, 0, 1]), + PartitionMode::Auto, + true, + )?); + let aggregate = Arc::new(AggregateExec::try_new( + AggregateMode::Single, + PhysicalGroupBy::new( + vec![(Arc::new(Column::new("a", 2)), "a".to_owned())], + vec![], + vec![], + ), + vec![Arc::new(Sum::new( + Arc::new(Column::new("ROW_NUMBER()", 1)), + "SUM(ROW_NUMBER())", + DataType::Int64, + ))], + vec![None], + join.clone(), + join.schema(), + )?) as Arc; + + let initial = get_plan_string(&aggregate); + + let expected_initial = [ + "AggregateExec: mode=Single, gby=[a@2 as a], aggr=[SUM(ROW_NUMBER())]", + " HashJoinExec: mode=Auto, join_type=LeftAnti, on=[(a@1, b@1)], filter=ROW_NUMBER()@0 < RANK@1, projection=[a@1, ROW_NUMBER()@0, a@1]", + " SortExec: expr=[ROW_NUMBER()@4 ASC,d@2 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[ROW_NUMBER()@5 as ROW_NUMBER(), a@0 as a, d@3 as d, d@3 as d, ROW_NUMBER()@5 as ROW_NUMBER()]", + " WindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " BoundedWindowAggExec: wdw=[RANK(): Ok(Field { name: \"RANK()\", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Linear]", + " 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(aggregate, &ConfigOptions::new())?; + + let expected = [ + "AggregateExec: mode=Single, gby=[a@2 as a], aggr=[SUM(ROW_NUMBER())]", + " HashJoinExec: mode=Auto, join_type=LeftAnti, on=[(a@1, b@0)], filter=ROW_NUMBER()@0 < RANK@1, projection=[a@1, ROW_NUMBER()@0, a@1]", + " ProjectionExec: expr=[ROW_NUMBER()@0 as ROW_NUMBER(), a@1 as a, ROW_NUMBER()@3 as ROW_NUMBER()]", + " SortExec: expr=[ROW_NUMBER()@3 ASC,d@2 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[ROW_NUMBER()@5 as ROW_NUMBER(), a@0 as a, d@3 as d, ROW_NUMBER()@5 as ROW_NUMBER()]", + " WindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " ProjectionExec: expr=[b@1 as b, RANK()@5 as RANK()]", + " BoundedWindowAggExec: wdw=[RANK(): Ok(Field { name: \"RANK()\", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Linear]", + " 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_optimize_projections_right_semi() -> Result<()> { + let csv_left = create_simple_csv_exec(); + let window = Arc::new(WindowAggExec::try_new( + vec![Arc::new(BuiltInWindowExpr::new( + Arc::new(RowNumber::new("ROW_NUMBER()".to_string(), &DataType::Int32)), + &[], + &[], + Arc::new(WindowFrame::new(None)), + ))], + csv_left, + vec![], + )?); + let projection = Arc::new(ProjectionExec::try_new( + vec![ + ( + Arc::new(Column::new("ROW_NUMBER()", 5)), + "ROW_NUMBER()".to_string(), + ), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ( + Arc::new(Column::new("ROW_NUMBER()", 5)), + "ROW_NUMBER()".to_string(), + ), + ], + window, + )?); + let sort = Arc::new(SortExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("ROW_NUMBER()", 4)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("d", 2)), + options: SortOptions::default(), + }, + ], + projection, + )); + + let csv_right = create_simple_csv_exec(); + let bounded_window = Arc::new(BoundedWindowAggExec::try_new( + vec![Arc::new(BuiltInWindowExpr::new( + Arc::new(rank("RANK()".to_string(), &DataType::Int32)), + &[], + &[], + Arc::new(WindowFrame::new(None)), + ))], + csv_right, + vec![], + InputOrderMode::Linear, + )?); + + let join = Arc::new(HashJoinExec::try_new( + sort, + bounded_window, + vec![(Arc::new(Column::new("a", 1)), Arc::new(Column::new("b", 1)))], + Some(JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("ROW_NUMBER()", 0)), + Operator::Lt, + Arc::new(Column::new("RANK", 1)), + )), + vec![ + ColumnIndex { + index: 4, + side: JoinSide::Left, + }, + ColumnIndex { + index: 5, + side: JoinSide::Right, + }, + ], + Schema::new(vec![ + Field::new("inter_rownumber", DataType::Int32, true), + Field::new("inter_avg_b", DataType::Float32, true), + ]), + )), + &JoinType::RightSemi, + Some(vec![1, 0, 1]), + PartitionMode::Auto, + true, + )?); + let aggregate = Arc::new(AggregateExec::try_new( + AggregateMode::Single, + PhysicalGroupBy::new( + vec![(Arc::new(Column::new("b", 2)), "b".to_owned())], + vec![], + vec![], + ), + vec![Arc::new(Sum::new( + Arc::new(Column::new("a", 1)), + "SUM(a)", + DataType::Int64, + ))], + vec![None], + join.clone(), + join.schema(), + )?) as Arc; + + let initial = get_plan_string(&aggregate); + + let expected_initial = [ + "AggregateExec: mode=Single, gby=[b@2 as b], aggr=[SUM(a)]", + " HashJoinExec: mode=Auto, join_type=RightSemi, on=[(a@1, b@1)], filter=ROW_NUMBER()@0 < RANK@1, projection=[b@1, a@0, b@1]", + " SortExec: expr=[ROW_NUMBER()@4 ASC,d@2 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[ROW_NUMBER()@5 as ROW_NUMBER(), a@0 as a, d@3 as d, d@3 as d, ROW_NUMBER()@5 as ROW_NUMBER()]", + " WindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " BoundedWindowAggExec: wdw=[RANK(): Ok(Field { name: \"RANK()\", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Linear]", + " 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(aggregate, &ConfigOptions::new())?; + + let expected = [ + "AggregateExec: mode=Single, gby=[b@2 as b], aggr=[SUM(a)]", + " HashJoinExec: mode=Auto, join_type=RightSemi, on=[(a@0, b@1)], filter=ROW_NUMBER()@0 < RANK@1, projection=[b@1, a@0, b@1]", + " ProjectionExec: expr=[a@0 as a, ROW_NUMBER()@2 as ROW_NUMBER()]", + " SortExec: expr=[ROW_NUMBER()@2 ASC,d@1 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[a@0 as a, d@3 as d, ROW_NUMBER()@5 as ROW_NUMBER()]", + " WindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " ProjectionExec: expr=[a@0 as a, b@1 as b, RANK()@5 as RANK()]", + " BoundedWindowAggExec: wdw=[RANK(): Ok(Field { name: \"RANK()\", data_type: Int32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Linear]", + " 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/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 416985983dfe..f677b486d9ce 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -20,7 +20,6 @@ use std::sync::Arc; use super::convert_first_last::OptimizeAggregateOrder; -use super::projection_pushdown::ProjectionPushdown; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; @@ -29,6 +28,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; @@ -104,8 +104,6 @@ impl PhysicalOptimizer { Arc::new(EnforceSorting::new()), // Run once after the local sorting requirement is changed Arc::new(OptimizeAggregateOrder::new()), - // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. - Arc::new(ProjectionPushdown::new()), // The CoalesceBatches rule will not influence the distribution and ordering of the // whole plan tree. Therefore, to avoid influencing other rules, it should run last. Arc::new(CoalesceBatches::new()), @@ -117,13 +115,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 to achieve 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()), // The PipelineChecker rule will reject non-runnable query plans that use // pipeline-breaking operators on infinite input(s). The rule generates a // diagnostic error message when this happens. It makes no changes to the diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs deleted file mode 100644 index 0190f35cc97b..000000000000 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ /dev/null @@ -1,2664 +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, ExecutionPlanProperties}; - -use arrow_schema::SchemaRef; -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{ - Transformed, TransformedResult, TreeNode, TreeNodeRecursion, -}; -use datafusion_common::{DataFusionError, JoinSide}; -use datafusion_physical_expr::expressions::{Column, Literal}; -use datafusion_physical_expr::{ - utils::collect_columns, 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).data() - } - - 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) - .data() - .map(Transformed::yes) - } 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)?.map_or_else( - || try_embed_to_hash_join(projection, hash_join), - |e| Ok(Some(e)), - )? - } 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(), - streaming_table.limit(), - ) - .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(|expr| { - Ok({ - if let Some(column) = expr.as_any().downcast_ref::() { - *column_ref_map.entry(column.clone()).or_default() += 1; - } - TreeNodeRecursion::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/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 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 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 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 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 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)))) -} - -/// Some projection can't be pushed down left input or right input of hash join because filter or on need may need some columns that won't be used in later. -/// By embed those projection to hash join, we can reduce the cost of build_batch_from_indices in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid unnecessary output creation. -fn try_embed_to_hash_join( - projection: &ProjectionExec, - hash_join: &HashJoinExec, -) -> Result>> { - // Collect all column indices from the given projection expressions. - let projection_index = collect_column_indices(projection.expr()); - - if projection_index.is_empty() { - return Ok(None); - }; - - // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. - // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of hash_join schema fields. - if projection_index.len() == projection_index.last().unwrap() + 1 - && projection_index.len() == hash_join.schema().fields().len() - { - return Ok(None); - } - - let new_hash_join = - Arc::new(hash_join.with_projection(Some(projection_index.to_vec()))?); - - // Build projection expressions for update_expr. Zip the projection_index with the new_hash_join output schema fields. - let embed_project_exprs = projection_index - .iter() - .zip(new_hash_join.schema().fields()) - .map(|(index, field)| { - ( - Arc::new(Column::new(field.name(), *index)) as Arc, - field.name().to_owned(), - ) - }) - .collect::>(); - - let mut new_projection_exprs = Vec::with_capacity(projection.expr().len()); - - for (expr, alias) in projection.expr() { - // update column index for projection expression since the input schema has been changed. - let Some(expr) = update_expr(expr, embed_project_exprs.as_slice(), false)? else { - return Ok(None); - }; - new_projection_exprs.push((expr, alias.clone())); - } - // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, but our projection_exprs in hash join just contain column, so we need to create the new projection to keep the original projection. - let new_projection = Arc::new(ProjectionExec::try_new( - new_projection_exprs, - new_hash_join.clone(), - )?); - if is_projection_removable(&new_projection) { - Ok(Some(new_hash_join)) - } else { - Ok(Some(new_projection)) - } -} - -/// Collect all column indices from the given projection expressions. -fn collect_column_indices(exprs: &[(Arc, String)]) -> Vec { - // Collect indices and remove duplicates. - let mut indexs = exprs - .iter() - .flat_map(|(expr, _)| collect_columns(expr)) - .map(|x| x.index()) - .collect::>() - .into_iter() - .collect::>(); - indexs.sort(); - indexs -} - -/// 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>> { - // TODO: currently if there is projection in HashJoinExec, we can't push down projection to left or right input. Maybe we can pushdown the mixed projection later. - if hash_join.contain_projection() { - return Ok(None); - } - - // 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.projection.clone(), - *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. -/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, -/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. -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(|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)), - ) - } - }) - .data(); - - 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 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)) - } - }) - .data() - .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 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 super::*; - use std::any::Any; - - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::physical_plan::FileScanConfig; - use crate::physical_plan::get_plan_string; - use crate::physical_plan::joins::StreamJoinPartitionMode; - - use arrow_schema::{DataType, Field, Schema, SortOptions}; - use datafusion_common::{JoinType, ScalarValue, Statistics}; - use datafusion_execution::object_store::ObjectStoreUrl; - use datafusion_execution::{SendableRecordBatchStream, TaskContext}; - use datafusion_expr::{ - ColumnarValue, Operator, ScalarUDF, ScalarUDFImpl, Signature, Volatility, - }; - use datafusion_physical_expr::expressions::{ - BinaryExpr, CaseExpr, CastExpr, NegativeExpr, - }; - use datafusion_physical_expr::ScalarFunctionExpr; - use datafusion_physical_plan::joins::PartitionMode; - use datafusion_physical_plan::streaming::PartitionStream; - - /// Mocked UDF - #[derive(Debug)] - struct DummyUDF { - signature: Signature, - } - - impl DummyUDF { - fn new() -> Self { - Self { - signature: Signature::variadic_any(Volatility::Immutable), - } - } - } - - impl ScalarUDFImpl for DummyUDF { - fn as_any(&self) -> &dyn Any { - self - } - - fn name(&self) -> &str { - "dummy_udf" - } - - fn signature(&self) -> &Signature { - &self.signature - } - - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(DataType::Int32) - } - - fn invoke(&self, _args: &[ColumnarValue]) -> Result { - unimplemented!("DummyUDF::invoke") - } - } - - #[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(ScalarUDF::new_from_impl(DummyUDF::new())), - 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, - )), - 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(ScalarUDF::new_from_impl(DummyUDF::new())), - 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, - )), - 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(ScalarUDF::new_from_impl(DummyUDF::new())), - 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, - )), - 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(ScalarUDF::new_from_impl(DummyUDF::new())), - 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, - )), - 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, - None, - )?; - 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_collect_column_indices() -> Result<()> { - let expr = Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 7)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a", 1)), - )), - )); - let column_indices = collect_column_indices(&[(expr, "b-(1+a)".to_string())]); - assert_eq!(column_indices, vec![1, 7]); - Ok(()) - } - - #[test] - fn test_hash_join_after_projection() -> Result<()> { - // sql like - // SELECT t1.c as c_from_left, t1.b as b_from_left, t1.a as a_from_left, t2.c as c_from_right FROM t1 JOIN t2 ON t1.b = t2.c WHERE t1.b - (1 + t2.a) <= t2.a + t1.c - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let join: Arc = Arc::new(HashJoinExec::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, - None, - PartitionMode::Auto, - true, - )?); - 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("c", 7)), "c_from_right".to_string()), - ], - join.clone(), - )?); - 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, c@7 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", " 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())?; - - // HashJoinExec only returns result after projection. Because there are some alias columns in the projection, the ProjectionExec is not removed. - let expected = ["ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@3 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " 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); - - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (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("c", 7)), "c".to_string()), - ], - join.clone(), - )?); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - // Comparing to the previous result, this projection don't have alias columns either change the order of output fields. So the ProjectionExec is removed. - let expected = ["HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " 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], preserve_partitioning=[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 = [ - "SortExec: expr=[b@2 ASC,c@0 + new_a@1 ASC], preserve_partitioning=[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(()) - } - - #[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_planner.rs b/datafusion/core/src/physical_planner.rs index 406196a59146..41e740f30a92 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -36,13 +36,10 @@ use crate::error::{DataFusionError, Result}; use crate::execution::context::{ExecutionProps, SessionState}; use crate::logical_expr::utils::generate_sort_key; use crate::logical_expr::{ - Aggregate, EmptyRelation, Join, Projection, Sort, TableScan, Unnest, Window, + expr_vec_fmt, Aggregate, EmptyRelation, Expr, Join, Limit, LogicalPlan, + Partitioning as LogicalPartitioning, PlanType, Projection, Repartition, Sort, + TableScan, Unnest, UserDefinedLogicalNode, Values, Window, }; -use crate::logical_expr::{ - Expr, LogicalPlan, Partitioning as LogicalPartitioning, PlanType, Repartition, - UserDefinedLogicalNode, -}; -use crate::logical_expr::{Limit, Values}; use crate::physical_expr::{create_physical_expr, create_physical_exprs}; use crate::physical_optimizer::optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; @@ -87,7 +84,6 @@ use datafusion_expr::expr::{ WindowFunction, }; use datafusion_expr::expr_rewriter::unnormalize_cols; -use datafusion_expr::expr_vec_fmt; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, StringifiedPlan, diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 3e5a0681589c..4af0eed76c56 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -658,17 +658,17 @@ 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![ - "CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@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", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@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", ]; let normalizer = ExplainNormalizer::new(); diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index d1202c83d526..f084c24f1c3f 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -39,3 +39,4 @@ path = "src/lib.rs" arrow = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } +indexmap = { workspace = true } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index da24f335b2f8..42f54c98fc13 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -28,7 +28,7 @@ use datafusion_expr::{ use std::fmt::Debug; use std::{any::Any, sync::Arc}; -use crate::physical_expr::PhysicalExpr; +use crate::physical_expr::{ExprMapping, PhysicalExpr}; use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; use self::utils::{down_cast_any_ref, ordering_fields}; @@ -150,6 +150,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 window expression with the given mapping. The `IndexMap` maps + /// existing expressions to modified expressions. Removed expressions have [`None`] value. + fn update_expression( + self: Arc, + _map: &ExprMapping, + ) -> Option> { + None + } } /// Physical aggregate expression of a UDAF. diff --git a/datafusion/physical-expr-common/src/expressions/column.rs b/datafusion/physical-expr-common/src/expressions/column.rs index 2cd52d6332fb..276ecab6f084 100644 --- a/datafusion/physical-expr-common/src/expressions/column.rs +++ b/datafusion/physical-expr-common/src/expressions/column.rs @@ -21,6 +21,8 @@ use std::any::Any; use std::hash::{Hash, Hasher}; use std::sync::Arc; +use crate::physical_expr::{down_cast_any_ref, PhysicalExpr}; + use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, @@ -28,8 +30,6 @@ use arrow::{ use datafusion_common::{internal_err, Result}; use datafusion_expr::ColumnarValue; -use crate::physical_expr::{down_cast_any_ref, PhysicalExpr}; - /// Represents the column at a given index in a RecordBatch #[derive(Debug, Hash, PartialEq, Eq, Clone)] pub struct Column { diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index a0f8bdf10377..3e4331323f43 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -20,16 +20,20 @@ use std::fmt::{Debug, Display}; use std::hash::{Hash, Hasher}; use std::sync::Arc; +use crate::expressions::column::Column; +use crate::sort_properties::SortProperties; +use crate::utils::scatter; + use arrow::array::BooleanArray; use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::ColumnarValue; -use crate::sort_properties::SortProperties; -use crate::utils::scatter; +use indexmap::IndexMap; /// See [create_physical_expr](https://docs.rs/datafusion/latest/datafusion/physical_expr/fn.create_physical_expr.html) /// for examples of creating `PhysicalExpr` from `Expr` @@ -208,3 +212,95 @@ pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { any } } + +/// Wrapper struct for `Arc` to use them as keys in a hash map. +#[derive(Debug, Clone)] +pub struct ExprWrapper(pub Arc); + +impl PartialEq for ExprWrapper { + fn eq(&self, other: &Self) -> bool { + self.0.eq(&other.0) + } +} + +impl Eq for ExprWrapper {} + +impl Hash for ExprWrapper { + fn hash(&self, state: &mut H) { + self.0.hash(state); + } +} + +#[derive(Debug)] +pub struct ExprMapping { + map: IndexMap>>, +} + +#[derive(Clone, Debug)] +pub enum ExprMappingValue { + Present(Arc), + Deleted, + Absent, +} + +impl ExprMapping { + pub fn new>( + schema_mapping: I, + ) -> ExprMapping { + let map = schema_mapping + .into_iter() + .map(|(source, target)| { + (ExprWrapper(Arc::new(source)), Some(Arc::new(target) as _)) + }) + .collect(); + Self { map } + } + + pub fn get( + &self, + key: Arc, + ) -> (Arc, ExprMappingValue) { + let key_wrapper = ExprWrapper(key); + if let Some(result) = self.map.get(&key_wrapper) { + if let Some(expr) = result { + (key_wrapper.0, ExprMappingValue::Present(expr.clone())) + } else { + (key_wrapper.0, ExprMappingValue::Deleted) + } + } else { + (key_wrapper.0, ExprMappingValue::Absent) + } + } + + pub fn update_expression( + &self, + expr: Arc, + ) -> Option> { + let (expr, value) = self.get(expr); + match value { + ExprMappingValue::Present(result) => Some(result), + ExprMappingValue::Deleted => None, + ExprMappingValue::Absent => { + let result = if expr.as_any().downcast_ref::().is_some() { + expr + } else { + expr.transform_up(|e| { + if e.as_any().downcast_ref::().is_some() { + let (e, value) = self.get(e); + return match value { + ExprMappingValue::Present(updated) => { + Ok(Transformed::yes(updated)) + } + _ => Ok(Transformed::no(e)), + }; + } + Ok(Transformed::no(e)) + }) + .unwrap() + .data + }; + Some(result) + } + } + } +} diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 3ce641c5aa46..2c53d0c608fa 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -300,7 +300,7 @@ mod tests { input_eq_properties: &EquivalenceProperties, ) -> Result<(RecordBatch, EquivalenceProperties)> { let input_schema = input_data.schema(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &input_schema)?; let output_schema = output_schema(&projection_mapping, &input_schema)?; let num_rows = input_data.num_rows(); diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 260610f23dc6..584a2a06ca56 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -49,16 +49,15 @@ impl ProjectionMapping { /// /// where `col("c + d")` means the column named `"c + d"`. pub fn try_new( - expr: &[(Arc, String)], + expr: Vec<(Arc, String)>, input_schema: &SchemaRef, ) -> Result { // Construct a map from the input expressions to the output expression of the projection: - expr.iter() + expr.into_iter() .enumerate() .map(|(expr_idx, (expression, name))| { - let target_expr = Arc::new(Column::new(name, expr_idx)) as _; + let target_expr = Arc::new(Column::new(&name, expr_idx)) as _; expression - .clone() .transform_down(|e| match e.as_any().downcast_ref::() { Some(col) => { // Sometimes, an expression and its name in the input_schema @@ -70,7 +69,7 @@ impl ProjectionMapping { if col.name() != matching_input_field.name() { return internal_err!("Input field name {} does not match with the projection expression {}", matching_input_field.name(),col.name()) - } + } let matching_input_column = Column::new(matching_input_field.name(), idx); Ok(Transformed::yes(Arc::new(matching_input_column))) @@ -599,7 +598,7 @@ mod tests { .into_iter() .map(|(expr, name)| (expr.clone(), name)) .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &schema)?; let output_schema = output_schema(&projection_mapping, &schema)?; let expected = expected @@ -675,7 +674,7 @@ mod tests { .into_iter() .map(|(expr, name)| (expr.clone(), name)) .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &schema)?; let output_schema = output_schema(&projection_mapping, &schema)?; let col_a_new = &col("a_new", &output_schema)?; @@ -856,7 +855,7 @@ mod tests { .into_iter() .map(|(expr, name)| (expr.clone(), name)) .collect::>(); - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &schema)?; let output_schema = output_schema(&projection_mapping, &schema)?; let col_a_plus_b_new = &col("a+b", &output_schema)?; @@ -1096,7 +1095,7 @@ mod tests { )?; let projection_mapping = - ProjectionMapping::try_new(&proj_exprs, &test_schema)?; + ProjectionMapping::try_new(proj_exprs, &test_schema)?; let projected_exprs = projection_mapping .iter() diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index c654208208df..e211eef6a507 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::hash::{Hash, Hasher}; use std::sync::Arc; use arrow_schema::{SchemaRef, SortOptions}; @@ -24,6 +23,7 @@ use itertools::Itertools; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; +use datafusion_physical_expr_common::physical_expr::ExprWrapper; use crate::equivalence::{ collapse_lex_req, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, @@ -1329,24 +1329,6 @@ fn updated_right_ordering_equivalence_class( } } -/// Wrapper struct for `Arc` to use them as keys in a hash map. -#[derive(Debug, Clone)] -struct ExprWrapper(Arc); - -impl PartialEq for ExprWrapper { - fn eq(&self, other: &Self) -> bool { - self.0.eq(&other.0) - } -} - -impl Eq for ExprWrapper {} - -impl Hash for ExprWrapper { - fn hash(&self, state: &mut H) { - self.0.hash(state); - } -} - #[cfg(test)] mod tests { use std::ops::Not; @@ -1386,7 +1368,7 @@ mod tests { (col_a.clone(), "a3".to_string()), (col_a.clone(), "a4".to_string()), ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &input_schema)?; let out_schema = output_schema(&projection_mapping, &input_schema)?; // a as a1, a as a2, a as a3, a as a3 @@ -1396,7 +1378,7 @@ mod tests { (col_a.clone(), "a3".to_string()), (col_a.clone(), "a4".to_string()), ]; - let projection_mapping = ProjectionMapping::try_new(&proj_exprs, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(proj_exprs, &input_schema)?; // a as a1, a as a2, a as a3, a as a3 let col_a1 = &col("a1", &out_schema)?; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index aef5aa7c00e7..ff4228c93882 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -49,7 +49,7 @@ pub use physical_expr::{ PhysicalExprRef, }; -pub use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +pub use datafusion_physical_expr_common::physical_expr::{ExprMapping, PhysicalExpr}; pub use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index dd9514c69a45..753d114717ba 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; -use crate::{LexOrderingRef, PhysicalExpr, PhysicalSortExpr}; +use crate::{ExprMapping, LexOrderingRef, PhysicalExpr, PhysicalSortExpr}; use arrow::array::{new_empty_array, Array, ArrayRef}; use arrow::compute::kernels::sort::SortColumn; @@ -128,6 +128,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 mapping. The [`IndexMap`] maps + /// existing expressions to modified expressions. Removed expressions have [`None`] value. + fn update_expression( + self: Arc, + _map: &ExprMapping, + ) -> 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 95376e7e69cd..1ac5fc953fd9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -389,7 +389,7 @@ impl AggregateExec { // construct a map from the input expression to the output expression of the Aggregation group by let projection_mapping = - ProjectionMapping::try_new(&group_by.expr, &input.schema())?; + ProjectionMapping::try_new(group_by.expr.clone(), &input.schema())?; let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index bf1ab8b73126..04043061270f 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -37,14 +37,15 @@ use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{internal_err, plan_err, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ - analyze, split_conjunction, AnalysisContext, ExprBoundaries, PhysicalExpr, + analyze, split_conjunction, AnalysisContext, ExprBoundaries, ExprMapping, + PhysicalExpr, }; use futures::stream::{Stream, StreamExt}; @@ -90,10 +91,7 @@ impl FilterExec { } } - pub fn with_default_selectivity( - mut self, - default_selectivity: u8, - ) -> Result { + pub fn with_default_selectivity(mut self, default_selectivity: u8) -> Result { if default_selectivity > 100 { return plan_err!("Default filter selectivity needs to be less than 100"); } @@ -287,6 +285,21 @@ impl ExecutionPlan for FilterExec { fn statistics(&self) -> Result { Self::statistics_helper(&self.input, self.predicate(), self.default_selectivity) } + + fn expressions(&self) -> Option>> { + Some(vec![self.predicate().clone()]) + } + + fn update_expressions( + self: Arc, + map: &ExprMapping, + ) -> Result>> { + let Some(new_predicate) = map.update_expression(self.predicate.clone()) else { + return internal_err!("Filter predicate cannot be empty"); + }; + FilterExec::try_new(new_predicate, self.input.clone()) + .map(|e| Some(Arc::new(e) as _)) + } } /// This function ensures that all bounds in the `ExprBoundaries` vector are diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index d3abedbe3806..006bea1526ed 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -543,7 +543,7 @@ impl HashJoinExec { let projection_exprs = project_index_to_exprs(projection, &schema); // construct a map from the input expressions to the output expression of the Projection let projection_mapping = - ProjectionMapping::try_new(&projection_exprs, &schema)?; + ProjectionMapping::try_new(projection_exprs, &schema)?; let out_schema = project_schema(&schema, Some(projection))?; if let Partitioning::Hash(exprs, part) = output_partitioning { let normalized_exprs = exprs diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 8d8a3e71031e..4967a70e5291 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -33,7 +33,8 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ - EquivalenceProperties, LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, + EquivalenceProperties, ExprMapping, LexOrdering, PhysicalSortExpr, + PhysicalSortRequirement, }; use futures::stream::TryStreamExt; @@ -426,6 +427,17 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { fn statistics(&self) -> Result { Ok(Statistics::new_unknown(&self.schema())) } + + fn expressions(&self) -> Option>> { + None + } + + fn update_expressions( + self: Arc, + _map: &ExprMapping, + ) -> Result>> { + Ok(None) + } } /// Extension trait provides an easy API to fetch various properties of diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index f72815c01a9e..e05cc9f26641 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -70,7 +70,7 @@ impl ProjectionExec { ) -> Result { let input_schema = input.schema(); - let fields: Result> = expr + let fields = expr .iter() .map(|(e, name)| { let mut field = Field::new( @@ -81,18 +81,16 @@ impl ProjectionExec { field.set_metadata( get_field_metadata(e, &input_schema).unwrap_or_default(), ); - Ok(field) }) - .collect(); - + .collect::>>()?; let schema = Arc::new(Schema::new_with_metadata( - fields?, + fields, input_schema.metadata().clone(), )); // construct a map from the input expressions to the output expression of the Projection - let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; + let projection_mapping = ProjectionMapping::try_new(expr.clone(), &input_schema)?; let cache = Self::compute_properties(&input, &projection_mapping, schema.clone())?; Ok(Self { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index e31fdc6ee2c2..e915735f664b 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -46,7 +46,9 @@ use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Res use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::{ + EquivalenceProperties, ExprMapping, PhysicalExpr, PhysicalSortExpr, +}; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; @@ -660,6 +662,33 @@ impl ExecutionPlan for RepartitionExec { fn statistics(&self) -> Result { self.input.statistics() } + + fn expressions(&self) -> Option>> { + if let Partitioning::Hash(exprs, _size) = self.partitioning() { + Some(exprs.clone()) + } else { + Some(vec![]) + } + } + + fn update_expressions( + self: Arc, + map: &ExprMapping, + ) -> Result>> { + let new_partitioning = if let Partitioning::Hash(exprs, size) = &self.partitioning + { + let updated_exprs = exprs + .iter() + .map(|expr| map.update_expression(expr.clone())) + .collect::>(); + Partitioning::Hash(updated_exprs.into_iter().flatten().collect(), *size) + } else { + self.partitioning.clone() + }; + + RepartitionExec::try_new(self.input.clone(), new_partitioning) + .map(|e| Some(Arc::new(e) as _)) + } } impl RepartitionExec { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index c684748bb29a..ab45647fffea 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -48,7 +48,7 @@ use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; use arrow_array::{Array, RecordBatchOptions, UInt32Array}; use arrow_schema::DataType; -use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_common::{exec_datafusion_err, exec_err, internal_err, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::{ @@ -56,7 +56,7 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr::{ExprMapping, LexOrdering, PhysicalExpr}; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; @@ -340,10 +340,7 @@ impl ExternalSorter { for spill in self.spills.drain(..) { if !spill.path().exists() { - return Err(DataFusionError::Internal(format!( - "Spill file {:?} does not exist", - spill.path() - ))); + return internal_err!("Spill file {:?} does not exist", spill.path()); } let stream = read_spill_as_stream(spill, self.schema.clone())?; streams.push(stream); @@ -726,7 +723,7 @@ fn read_spill(sender: Sender>, path: &Path) -> Result<()> { for batch in reader { sender .blocking_send(batch.map_err(Into::into)) - .map_err(|e| DataFusionError::Execution(format!("{e}")))?; + .map_err(|e| exec_datafusion_err!("{e}"))?; } Ok(()) } @@ -992,6 +989,38 @@ impl ExecutionPlan for SortExec { fn statistics(&self) -> Result { self.input.statistics() } + + fn expressions(&self) -> Option>> { + Some( + self.expr() + .iter() + .map(|sort_expr| sort_expr.expr.clone()) + .collect(), + ) + } + + fn update_expressions( + self: Arc, + map: &ExprMapping, + ) -> Result>> { + let new_sort_exprs = self + .expr() + .iter() + .filter_map(|sort_expr| { + map.update_expression(sort_expr.expr.clone()) + .map(|new_sort_expr| PhysicalSortExpr { + expr: new_sort_expr, + options: sort_expr.options, + }) + }) + .collect::>(); + + Ok(Some(Arc::new( + SortExec::new(new_sort_exprs, self.input.clone()) + .with_preserve_partitioning(self.preserve_partitioning) + .with_fetch(self.fetch), + ))) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 88c6c312b94b..68232d61b0f2 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -32,7 +32,7 @@ use crate::{ use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr::{ExprMapping, PhysicalExpr, PhysicalSortRequirement}; use log::{debug, trace}; @@ -257,6 +257,37 @@ impl ExecutionPlan for SortPreservingMergeExec { fn statistics(&self) -> Result { self.input.statistics() } + + fn expressions(&self) -> Option>> { + Some( + self.expr() + .iter() + .map(|sort_expr| sort_expr.expr.clone()) + .collect(), + ) + } + + fn update_expressions( + self: Arc, + map: &ExprMapping, + ) -> Result>> { + let new_sort_exprs = self + .expr + .iter() + .filter_map(|sort_expr| { + map.update_expression(sort_expr.expr.clone()) + .map(|new_sort_expr| PhysicalSortExpr { + expr: new_sort_expr, + options: sort_expr.options, + }) + }) + .collect::>(); + + Ok(Some(Arc::new( + SortPreservingMergeExec::new(new_sort_exprs, self.input.clone()) + .with_fetch(self.fetch), + ))) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index cff91283eb6e..1264bf5a6e29 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -51,14 +51,16 @@ use datafusion_common::utils::{ evaluate_partition_ranges, get_arrayref_at_indices, get_at_indices, get_record_batch_at_indices, get_row_at_idx, }; -use datafusion_common::{arrow_datafusion_err, exec_err, DataFusionError, Result}; +use datafusion_common::{ + arrow_datafusion_err, exec_datafusion_err, exec_err, DataFusionError, Result, +}; use datafusion_execution::TaskContext; use datafusion_expr::window_state::{PartitionBatchState, WindowAggState}; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::{ExprMapping, PhysicalExpr, PhysicalSortRequirement}; use ahash::RandomState; use futures::stream::Stream; @@ -333,6 +335,43 @@ impl ExecutionPlan for BoundedWindowAggExec { total_byte_size: Precision::Absent, }) } + + fn expressions(&self) -> Option>> { + let mut all_exprs = self + .window_expr() + .iter() + .flat_map(|window_expr| window_expr.expressions()) + .collect::>(); + all_exprs.extend(self.partition_keys.clone()); + Some(all_exprs) + } + + fn update_expressions( + self: Arc, + map: &ExprMapping, + ) -> Result>> { + let Some(new_window_exprs) = self + .window_expr() + .iter() + .map(|window_expr| window_expr.clone().update_expression(map)) + .collect::>>() + else { + return Ok(None); + }; + let new_keys = self + .partition_keys + .iter() + .filter_map(|key| map.update_expression(key.clone())) + .collect(); + + BoundedWindowAggExec::try_new( + new_window_exprs, + self.input.clone(), + new_keys, + self.input_order_mode.clone(), + ) + .map(|e| Some(Arc::new(e) as _)) + } } /// Trait that specifies how we search for (or calculate) partitions. It has two @@ -1157,8 +1196,7 @@ fn get_aggregate_result_out_column( "Generated row number should be {len_to_show}, it is {running_length}" ); } - result - .ok_or_else(|| DataFusionError::Execution("Should contain something".to_string())) + result.ok_or_else(|| exec_datafusion_err!("Should contain something")) } /// Constructs a batch from the last row of batch in the argument. diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 1507902c22ea..fe9e65cfb0b9 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -43,7 +43,7 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr::{ExprMapping, PhysicalSortRequirement}; use futures::{ready, Stream, StreamExt}; @@ -263,6 +263,38 @@ impl ExecutionPlan for WindowAggExec { total_byte_size: Precision::Absent, }) } + + fn expressions(&self) -> Option>> { + let mut all_exprs = self + .window_expr() + .iter() + .flat_map(|window_expr| window_expr.expressions()) + .collect::>(); + all_exprs.extend(self.partition_keys.clone()); + Some(all_exprs) + } + + fn update_expressions( + self: Arc, + map: &ExprMapping, + ) -> Result>> { + let Some(new_window_exprs) = self + .window_expr() + .iter() + .map(|window_expr| window_expr.clone().update_expression(map)) + .collect::>>() + else { + return Ok(None); + }; + let new_keys = self + .partition_keys + .iter() + .filter_map(|key| map.update_expression(key.clone())) + .collect(); + + WindowAggExec::try_new(new_window_exprs, self.input.clone(), new_keys) + .map(|e| Some(Arc::new(e) as _)) + } } fn create_schema( diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 983f8a085ba9..873dc6ba2d0e 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3962,14 +3962,13 @@ physical_plan 01)GlobalLimitExec: skip=10, fetch=3 02)--AggregateExec: mode=Final, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] +04)------AggregateExec: mode=Partial, gby=[c3@1 as c3, c2@0 as c2], aggr=[], lim=[13] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------ProjectionExec: expr=[c3@1 as c3, c2@0 as c2] -07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -08)--------------CoalescePartitionsExec -09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +06)----------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +07)------------CoalescePartitionsExec +08)--------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query II SELECT DISTINCT c3, c2 FROM aggregate_test_100 group by c2, c3 limit 3 offset 10; diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 1ff108cf6c5f..39c1364b758f 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -736,11 +736,11 @@ physical_plan 05)----CrossJoinExec 06)------CoalescePartitionsExec 07)--------CoalesceBatchesExec: target_batch_size=8182 -08)----------FilterExec: val@0 < 2 -09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)--------------WorkTableExec: name=recursive_cte -11)------ProjectionExec: expr=[2 as val] -12)--------PlaceholderRowExec +08)----------ProjectionExec: expr=[] +09)------------FilterExec: val@0 < 2 +10)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)----------------WorkTableExec: name=recursive_cte +12)------PlaceholderRowExec # Test issue: https://github.com/apache/datafusion/issues/9794 # Non-recursive term and recursive term have different types diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 92c537f975ad..0a1dc1f54dc4 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,8 +43,8 @@ logical_plan 02)--Filter: aggregate_test_100.c2 > Int8(10) 03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -01)ProjectionExec: expr=[c1@0 as c1] -02)--CoalesceBatchesExec: target_batch_size=8192 +01)CoalesceBatchesExec: target_batch_size=8192 +02)--ProjectionExec: expr=[c1@0 as c1] 03)----FilterExec: c2@1 > 10 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true @@ -249,11 +249,10 @@ physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after OptimizeProjections SAME TEXT AS ABOVE physical_plan after PipelineChecker 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]:)]] @@ -306,13 +305,12 @@ physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 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 after PipelineChecker SAME TEXT AS ABOVE physical_plan 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -343,13 +341,12 @@ physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10 02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 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 after PipelineChecker SAME TEXT AS ABOVE physical_plan 01)GlobalLimitExec: skip=0, fetch=10 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 43bbf6bed643..67098d96eb31 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2875,8 +2875,8 @@ physical_plan 01)SortExec: expr=[sn@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] 03)----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]] -04)------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] -05)--------CoalesceBatchesExec: target_batch_size=8192 +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] 06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------------MemoryExec: partitions=1, partition_sizes=[1] @@ -4026,8 +4026,8 @@ logical_plan 09)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] 10)----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -01)ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] -02)--CoalesceBatchesExec: target_batch_size=2 +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] 04)------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] 05)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 8de8c478fbc4..c8cde5929355 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -99,8 +99,8 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=10 02)--SortPreservingMergeExec: [a2@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 -03)----ProjectionExec: expr=[a@0 as a2, b@1 as b] -04)------CoalesceBatchesExec: target_batch_size=8192 +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------ProjectionExec: expr=[a@0 as a2, b@1 as b] 05)--------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true 07)----------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 0c45e3ffbf69..7e09ebc40979 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2031,8 +2031,8 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id] 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[t2_id@0 as t2_id] -05)--------CoalesceBatchesExec: target_batch_size=2 +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------ProjectionExec: expr=[t2_id@0 as t2_id] 06)----------FilterExec: t2_int@1 > 1 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 2c65b1da4474..8346c8bd3bb2 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -370,9 +370,9 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------ProjectionExec: expr=[] -06)----------GlobalLimitExec: skip=6, fetch=3 -07)------------CoalesceBatchesExec: target_batch_size=8192 +05)--------GlobalLimitExec: skip=6, fetch=3 +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------ProjectionExec: expr=[] 08)--------------FilterExec: a@0 > 3 09)----------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 5483750c6a03..ecc0b8bf99bb 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -770,8 +770,8 @@ physical_plan 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true 10)----------CoalesceBatchesExec: target_batch_size=8192 11)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -12)--------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -13)----------------CoalesceBatchesExec: target_batch_size=8192 +12)--------------CoalesceBatchesExec: target_batch_size=8192 +13)----------------ProjectionExec: expr=[p_partkey@0 as p_partkey] 14)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 16)----------------------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/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 4a9fb38e7db1..ec5792c84df0 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -187,18 +187,17 @@ logical_plan 06)--------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int)@0, t1_id@2] -04)------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] -05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -09)----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -10)------CoalesceBatchesExec: target_batch_size=2 -11)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -12)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] +03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t1_id@2] +04)------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +08)--------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +11)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -221,8 +220,8 @@ logical_plan 06)--------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Float64)) AS SUM(t2.t2_int * Float64(1))]] 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] 03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int * Float64(1)) + Int64(1)@0, t1_id@2] 04)------ProjectionExec: expr=[SUM(t2.t2_int * Float64(1))@1 + 1 as SUM(t2.t2_int * Float64(1)) + Int64(1), t2_id@0 as t2_id] 05)--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] @@ -255,8 +254,8 @@ logical_plan 06)--------Aggregate: groupBy=[[t2.t2_id, Utf8("a")]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] 07)----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] 03)----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)], projection=[t1_id@0, SUM(t2.t2_int)@1] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 @@ -292,20 +291,19 @@ logical_plan 07)----------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] 08)------------TableScan: t2 projection=[t2_id, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)], projection=[SUM(t2.t2_int)@0, t1_id@2] -04)------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------FilterExec: SUM(t2.t2_int)@1 < 3 -07)------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -08)--------------CoalesceBatchesExec: target_batch_size=2 -09)----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -10)------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -11)--------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -12)------CoalesceBatchesExec: target_batch_size=2 -13)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -14)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] +03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t1_id@2] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------FilterExec: SUM(t2.t2_int)@1 < 3 +06)----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +09)----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +10)------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +11)------CoalesceBatchesExec: target_batch_size=2 +12)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +13)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index f2c14f2628ee..024d32f2ae8d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -56,8 +56,9 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), AVG(lineitem.l_quantity), AVG(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(*)] 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as {lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)|{Decimal128(Some(1),20,0) - lineitem.l_discount|{lineitem.l_discount}|{Decimal128(Some(1),20,0)}}|{lineitem.l_extendedprice}}, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------FilterExec: l_shipdate@6 <= 10471 -11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false +10)------------------ProjectionExec: expr=[l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] +11)--------------------FilterExec: l_shipdate@6 <= 10471 +12)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false query TTRRRRRRRI select diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/q10.slt.part index b732610de095..587be5861f1f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -94,14 +94,14 @@ physical_plan 22)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false 23)------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -25)----------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -26)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] 27)--------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 28)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false 29)----------------------------CoalesceBatchesExec: target_batch_size=8192 30)------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -31)--------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -32)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 33)------------------------------------FilterExec: l_returnflag@3 = R 34)--------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false 35)--------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index c8b9bf988738..9799ef0a88c3 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -98,8 +98,8 @@ physical_plan 22)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false 23)----------------------CoalesceBatchesExec: target_batch_size=8192 24)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -25)--------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -26)----------------------------CoalesceBatchesExec: target_batch_size=8192 +25)--------------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] 27)------------------------------FilterExec: n_name@1 = GERMANY 28)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 29)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false @@ -122,8 +122,8 @@ physical_plan 46)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false 47)--------------------CoalesceBatchesExec: target_batch_size=8192 48)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -49)------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -50)--------------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------CoalesceBatchesExec: target_batch_size=8192 +50)--------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] 51)----------------------------FilterExec: n_name@1 = GERMANY 52)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 53)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/q12.slt.part index 5f1b344f784d..b3943f0f0f26 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -70,8 +70,8 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] 14)--------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 8766 AND l_receiptdate@3 < 9131 15)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false 16)------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index f19db720fb2c..e845bf2af613 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -72,8 +72,8 @@ physical_plan 16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false 17)------------------------CoalesceBatchesExec: target_batch_size=8192 18)--------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -19)----------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -20)------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] 21)--------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% 22)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/q14.slt.part index 3d598fdfb63a..d53fe0b3177e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -49,8 +49,8 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -09)----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -10)------------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 11)--------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 12)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false 13)------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index b8bb31288781..00e992316629 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -88,8 +88,8 @@ physical_plan 15)--------------------CoalesceBatchesExec: target_batch_size=8192 16)----------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 17)------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -18)--------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 20)------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false 22)--------CoalesceBatchesExec: target_batch_size=8192 @@ -102,8 +102,8 @@ physical_plan 29)----------------------CoalesceBatchesExec: target_batch_size=8192 30)------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 31)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -32)----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -33)------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 34)--------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 35)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 2b01980f0e6f..ccd985aced1c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -95,8 +95,8 @@ physical_plan 27)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], has_header=false 28)----------------------------CoalesceBatchesExec: target_batch_size=8192 29)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -30)--------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] -31)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] 32)------------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% 33)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 34)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/q17.slt.part index 19fd2375f66c..4ca2eee20910 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -64,8 +64,8 @@ physical_plan 11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false 12)----------------CoalesceBatchesExec: target_batch_size=8192 13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)--------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -15)----------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] 16)------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX 17)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 18)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index a36a45cee5d0..5a6a494da011 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -91,8 +91,8 @@ physical_plan 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 24)------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false -25)----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -26)------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------CoalesceBatchesExec: target_batch_size=8192 +26)------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] 27)--------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 28)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] 29)------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index 9a49fc424eb6..51150d6f2f50 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -72,8 +72,8 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) 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 Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) 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 Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -09)----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] -10)------------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] 11)--------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON 12)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false 13)------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index 1b1293cd250e..d2fb28b8e7de 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -105,7 +105,7 @@ physical_plan 03)----SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] 05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@1)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 09)----------------CoalesceBatchesExec: target_batch_size=8192 @@ -113,76 +113,74 @@ physical_plan 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@5, s_acctbal@6, s_comment@7, ps_supplycost@8, n_name@10, n_regionkey@11] +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@5, n_nationkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@3, s_address@4, s_phone@6, s_acctbal@7, s_comment@8, ps_supplycost@2, n_name@10, n_regionkey@11] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 -17)--------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@3 as s_name, s_address@4 as s_address, s_nationkey@5 as s_nationkey, s_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@2 as ps_supplycost] -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_supplycost@3, s_name@5, s_address@6, s_nationkey@7, s_phone@8, s_acctbal@9, s_comment@10] -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 -22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] -24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@5], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_supplycost@3, s_name@5, s_address@6, s_nationkey@7, s_phone@8, s_acctbal@9, s_comment@10] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 +21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] +23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 26)--------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] -27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS -29)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false -31)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -33)--------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -34)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)----------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -36)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -37)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false -38)----------------------------CoalesceBatchesExec: target_batch_size=8192 -39)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -40)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -41)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false -42)--------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -44)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -45)--------------------------CoalesceBatchesExec: target_batch_size=8192 -46)----------------------------FilterExec: r_name@1 = EUROPE -47)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false -49)------------CoalesceBatchesExec: target_batch_size=8192 -50)--------------RepartitionExec: partitioning=Hash([ps_partkey@1, MIN(partsupp.ps_supplycost)@0], 4), input_partitions=4 -51)----------------ProjectionExec: expr=[MIN(partsupp.ps_supplycost)@1 as MIN(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -52)------------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] -53)--------------------CoalesceBatchesExec: target_batch_size=8192 -54)----------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -55)------------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] -56)--------------------------CoalesceBatchesExec: target_batch_size=8192 -57)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -58)------------------------------CoalesceBatchesExec: target_batch_size=8192 -59)--------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -60)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -61)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -62)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -63)----------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -64)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -65)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -66)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -67)------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -68)--------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -69)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -70)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -71)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -72)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -73)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -74)----------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -75)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -76)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -77)------------------------------CoalesceBatchesExec: target_batch_size=8192 -78)--------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -79)----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -80)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -81)--------------------------------------FilterExec: r_name@1 = EUROPE -82)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -83)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +27)----------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS +28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false +30)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +32)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +33)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +35)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +36)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false +37)----------------------------CoalesceBatchesExec: target_batch_size=8192 +38)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +39)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +40)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +41)--------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +43)------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +45)----------------------------FilterExec: r_name@1 = EUROPE +46)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +47)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +48)------------CoalesceBatchesExec: target_batch_size=8192 +49)--------------RepartitionExec: partitioning=Hash([ps_partkey@0, MIN(partsupp.ps_supplycost)@1], 4), input_partitions=4 +50)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +51)------------------CoalesceBatchesExec: target_batch_size=8192 +52)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +53)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +54)------------------------CoalesceBatchesExec: target_batch_size=8192 +55)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +56)----------------------------CoalesceBatchesExec: target_batch_size=8192 +57)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +58)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +59)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +60)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +61)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +62)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +63)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +64)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +65)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +66)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +67)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +68)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +69)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +70)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +71)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +72)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +73)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +74)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +75)----------------------------CoalesceBatchesExec: target_batch_size=8192 +76)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +77)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +78)----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +79)------------------------------------FilterExec: r_name@1 = EUROPE +80)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +81)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/q20.slt.part index 6b772b2ab00f..9e8f27ffc753 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -95,8 +95,8 @@ physical_plan 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false 13)----------------CoalesceBatchesExec: target_batch_size=8192 14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -15)--------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -16)----------------------CoalesceBatchesExec: target_batch_size=8192 +15)--------------------CoalesceBatchesExec: target_batch_size=8192 +16)----------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] 17)------------------------FilterExec: n_name@1 = CANADA 18)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 19)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false @@ -113,8 +113,8 @@ physical_plan 30)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false 31)------------------------CoalesceBatchesExec: target_batch_size=8192 32)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -33)----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -34)------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------------------CoalesceBatchesExec: target_batch_size=8192 +34)------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] 35)--------------------------------FilterExec: p_name@1 LIKE forest% 36)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 37)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false @@ -123,8 +123,8 @@ physical_plan 40)--------------------CoalesceBatchesExec: target_batch_size=8192 41)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 42)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] -43)--------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] -44)----------------------------CoalesceBatchesExec: target_batch_size=8192 +43)--------------------------CoalesceBatchesExec: target_batch_size=8192 +44)----------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] 45)------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 46)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index b536dd281eca..0279f15541d9 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -119,20 +119,20 @@ physical_plan 27)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false 28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -30)--------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] 32)------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 33)--------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false 34)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 35)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -36)------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] -37)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)--------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] 38)----------------------------------------------FilterExec: o_orderstatus@1 = F 39)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false 40)------------------------------CoalesceBatchesExec: target_batch_size=8192 41)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -42)----------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -43)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] 44)--------------------------------------FilterExec: n_name@1 = SAUDI ARABIA 45)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 46)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false @@ -141,8 +141,8 @@ physical_plan 49)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], has_header=false 50)------------------CoalesceBatchesExec: target_batch_size=8192 51)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -52)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -53)------------------------CoalesceBatchesExec: target_batch_size=8192 +52)----------------------CoalesceBatchesExec: target_batch_size=8192 +53)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] 54)--------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 55)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/q22.slt.part index 21fc7a2bb002..c970d678ebd1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -83,27 +83,28 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] 08)--------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 -11)--------------------CoalescePartitionsExec -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) -18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], has_header=false -20)--------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -22)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], has_header=false -23)--------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] -24)----------------------CoalescePartitionsExec -25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] -26)--------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] +10)------------------ProjectionExec: expr=[c_phone@0 as c_phone, c_acctbal@1 as c_acctbal] +11)--------------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 +12)----------------------CoalescePartitionsExec +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) +19)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +20)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], has_header=false +21)----------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +23)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_custkey], has_header=false +24)----------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] +25)------------------------CoalescePartitionsExec +26)--------------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] 27)----------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) -29)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +28)------------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] +29)--------------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) +30)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false query TIR diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index 85dd95888613..9d93ca355d8a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -75,8 +75,8 @@ physical_plan 14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -17)--------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] 19)------------------------------------FilterExec: c_mktsegment@1 = BUILDING 20)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 21)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false @@ -87,8 +87,8 @@ physical_plan 26)------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false 27)--------------------CoalesceBatchesExec: target_batch_size=8192 28)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -29)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -30)--------------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 31)----------------------------FilterExec: l_shipdate@3 > 9204 32)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/q4.slt.part index 6f7b6e347773..ce855dd022d6 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -64,14 +64,14 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] 14)--------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 15)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false 16)------------------CoalesceBatchesExec: target_batch_size=8192 17)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -18)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -19)------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] 20)--------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 21)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/q5.slt.part index ec2edc784798..eb7924b8073d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -98,8 +98,8 @@ physical_plan 29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false 30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -32)------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] 34)----------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 35)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false 36)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 @@ -115,8 +115,8 @@ physical_plan 46)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false 47)------------------CoalesceBatchesExec: target_batch_size=8192 48)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -49)----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -50)------------------------CoalesceBatchesExec: target_batch_size=8192 +49)----------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] 51)--------------------------FilterExec: r_name@1 = ASIA 52)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 53)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/q6.slt.part index e54b3c1ccd03..f3992145a8c1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q6.slt.part @@ -38,8 +38,8 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] -05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -06)----------CoalesceBatchesExec: target_batch_size=8192 +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] 07)------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2 08)--------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 1e85ad9418c2..749ac193794a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -126,8 +126,8 @@ physical_plan 34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] 35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -37)------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -38)--------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] 39)----------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL 40)------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 41)--------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false @@ -157,8 +157,8 @@ physical_plan 65)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false 66)--------------------CoalesceBatchesExec: target_batch_size=8192 67)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -68)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -69)--------------------------CoalesceBatchesExec: target_batch_size=8192 +68)------------------------CoalesceBatchesExec: target_batch_size=8192 +69)--------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] 70)----------------------------FilterExec: r_name@1 = AMERICA 71)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 72)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index f45170338303..26f91b19551d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -105,8 +105,8 @@ physical_plan 27)----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] 28)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 29)--------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -30)----------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] 32)--------------------------------------------------------------FilterExec: p_name@1 LIKE %green% 33)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 34)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 36f024961875..ccad4cbab890 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -278,7 +278,7 @@ logical_plan physical_plan 01)UnionExec 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] +03)----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@1), (name@1, name@0)] 04)------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 @@ -286,8 +286,8 @@ physical_plan 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------MemoryExec: partitions=1, partition_sizes=[1] 10)------CoalesceBatchesExec: target_batch_size=2 -11)--------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -12)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +11)--------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@1, name@0], 4), input_partitions=4 +12)----------ProjectionExec: expr=[name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] 13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 14)--------------MemoryExec: partitions=1, partition_sizes=[1] 15)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index be1517aa75c1..4aee8a984366 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1435,8 +1435,8 @@ logical_plan 05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] 06)----------TableScan: aggregate_test_100 projection=[c1, c2, c9] physical_plan -01)ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2] 03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] 05)--------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] @@ -1517,23 +1517,24 @@ logical_plan 11)--------------------WindowAggr: windowExpr=[[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 12)----------------------TableScan: null_cases projection=[c1, c2, c3] physical_plan -01)ProjectionExec: expr=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@18 as a, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@18 as b, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@3 as c, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@11 as d, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@7 as e, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@3 as f, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@11 as g, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as h, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as i, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as j, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as k, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 as l, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as m, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@15 as n, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as o, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as p, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@20 as a1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@20 as b1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@5 as c1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@13 as d1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@9 as e1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@5 as f1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@13 as g1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as h1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as j1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as k1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as l1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 as m1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as n1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as o1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@21 as h11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@21 as j11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@6 as k11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@14 as l11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@10 as m11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@6 as n11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@14 as o11] +01)ProjectionExec: expr=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@16 as a, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@16 as b, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@1 as c, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@9 as d, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@5 as e, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@1 as f, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@9 as g, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as h, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as i, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@10 as j, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as k, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as l, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@15 as m, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as n, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@0 as o, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@14 as p, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@18 as a1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@18 as b1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@3 as c1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@11 as d1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@7 as e1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@3 as f1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@11 as g1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as h1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as j1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as k1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@10 as l1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as m1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as n1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@10 as o1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@19 as h11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@19 as j11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@4 as k11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@12 as l11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@8 as m11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@4 as n11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@12 as o11] 02)--GlobalLimitExec: skip=0, fetch=5 -03)----WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] -04)------ProjectionExec: expr=[c1@0 as c1, c3@2 as c3, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@4 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@6 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@7 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@8 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@9 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@10 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@11 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@12 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@14 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@15 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@18 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -05)--------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] -07)------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -08)--------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC], preserve_partitioning=[false] -09)----------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -10)------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC], preserve_partitioning=[false] -11)--------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }] -12)----------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] -13)------------------------SortExec: expr=[c3@2 DESC NULLS LAST], preserve_partitioning=[false] -14)--------------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] -15)----------------------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -16)------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST], preserve_partitioning=[false] -17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true +03)----ProjectionExec: expr=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@3 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@5 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@6 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@7 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@8 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@9 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@10 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@11 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@12 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@13 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@14 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@15 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@18 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@19 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@20 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@21 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING] +04)------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] +05)--------ProjectionExec: expr=[c1@0 as c1, c3@2 as c3, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@4 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@6 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@7 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@8 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@9 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@10 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@11 as SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@12 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@14 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@15 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@18 as SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +06)----------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +07)------------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] +08)--------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +09)----------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC], preserve_partitioning=[false] +10)------------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +11)--------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC], preserve_partitioning=[false] +12)----------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }] +13)------------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] +14)--------------------------SortExec: expr=[c3@2 DESC NULLS LAST], preserve_partitioning=[false] +15)----------------------------WindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] +16)------------------------------BoundedWindowAggExec: wdw=[SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +17)--------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST], preserve_partitioning=[false] +18)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -1603,8 +1604,8 @@ logical_plan 04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] 05)--------TableScan: aggregate_test_100 projection=[c1, c9] physical_plan -01)ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] 03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC], preserve_partitioning=[false] @@ -1647,8 +1648,8 @@ logical_plan 04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] 05)--------TableScan: aggregate_test_100 projection=[c1, c9] physical_plan -01)ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum2] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum2] 03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC], preserve_partitioning=[false] @@ -1693,8 +1694,8 @@ logical_plan 06)----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS {aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}, aggregate_test_100.c2, aggregate_test_100.c3, aggregate_test_100.c9 07)------------TableScan: aggregate_test_100 projection=[c2, c3, c4, c9] physical_plan -01)ProjectionExec: expr=[c3@1 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c3@1 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2] 03)----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] 04)------ProjectionExec: expr=[{aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}@0 as {aggregate_test_100.c3 + aggregate_test_100.c4|{aggregate_test_100.c4}|{aggregate_test_100.c3}}, c3@2 as c3, c9@3 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] @@ -1752,8 +1753,8 @@ physical_plan 08)--------------CoalesceBatchesExec: target_batch_size=4096 09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 10)------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -11)--------------------ProjectionExec: expr=[c1@0 as c1] -12)----------------------CoalesceBatchesExec: target_batch_size=4096 +11)--------------------CoalesceBatchesExec: target_batch_size=4096 +12)----------------------ProjectionExec: expr=[c1@0 as c1] 13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true @@ -2092,7 +2093,6 @@ physical_plan 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true - query IIIII SELECT c9, SUM(c9) OVER(PARTITION BY c1, c2 ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING) as sum1, @@ -2137,8 +2137,8 @@ logical_plan 11)--------------------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c8, aggregate_test_100.c9, aggregate_test_100.c1 AS c1_alias 12)----------------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] physical_plan -01)ProjectionExec: expr=[c9@1 as c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@1 as c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] 03)----BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 04)------ProjectionExec: expr=[c2@0 as c2, c9@2 as c9, c1_alias@3 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] 05)--------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] @@ -2183,8 +2183,8 @@ logical_plan 07)------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 08)--------------TableScan: aggregate_test_100 projection=[c1, c2, c9, c12] physical_plan -01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] 03)----SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST], preserve_partitioning=[false] 04)------ProjectionExec: expr=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum1, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING@4 as sum2, c9@1 as c9] 05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING: Ok(Field { name: "SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(3)), is_causal: false }], mode=[Sorted] @@ -2550,8 +2550,8 @@ logical_plan 09)----------------Projection: CAST(annotated_data_finite.desc_col AS Int64) AS {CAST(annotated_data_finite.desc_col AS Int64)|{annotated_data_finite.desc_col}}, CAST(annotated_data_finite.inc_col AS Int64) AS {CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}}, annotated_data_finite.ts, annotated_data_finite.inc_col, annotated_data_finite.desc_col 10)------------------TableScan: annotated_data_finite projection=[ts, inc_col, desc_col] physical_plan -01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, sum3@2 as sum3, min1@3 as min1, min2@4 as min2, min3@5 as min3, max1@6 as max1, max2@7 as max2, max3@8 as max3, cnt1@9 as cnt1, cnt2@10 as cnt2, sumr1@11 as sumr1, sumr2@12 as sumr2, sumr3@13 as sumr3, minr1@14 as minr1, minr2@15 as minr2, minr3@16 as minr3, maxr1@17 as maxr1, maxr2@18 as maxr2, maxr3@19 as maxr3, cntr1@20 as cntr1, cntr2@21 as cntr2, sum4@22 as sum4, cnt3@23 as cnt3] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, sum3@2 as sum3, min1@3 as min1, min2@4 as min2, min3@5 as min3, max1@6 as max1, max2@7 as max2, max3@8 as max3, cnt1@9 as cnt1, cnt2@10 as cnt2, sumr1@11 as sumr1, sumr2@12 as sumr2, sumr3@13 as sumr3, minr1@14 as minr1, minr2@15 as minr2, minr3@16 as minr3, maxr1@17 as maxr1, maxr2@18 as maxr2, maxr3@19 as maxr3, cntr1@20 as cntr1, cntr2@21 as cntr2, sum4@22 as sum4, cnt3@23 as cnt3] 03)----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC], preserve_partitioning=[false] 04)------ProjectionExec: expr=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as sum1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@14 as sum2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@15 as sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as max3, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@22 as cnt1, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@23 as cnt2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@2 as sumr1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@3 as sumr2, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@6 as minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@7 as minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as maxr3, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@11 as cntr1, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@12 as cntr2, SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@24 as sum4, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@25 as cnt3, inc_col@1 as inc_col] 05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 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(8)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ROWS BETWEEN 8 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(8)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] @@ -2712,8 +2712,8 @@ logical_plan 07)------------Projection: CAST(annotated_data_finite.inc_col AS Int64) AS {CAST(annotated_data_finite.inc_col AS Int64)|{annotated_data_finite.inc_col}}, CAST(annotated_data_finite.inc_col AS Float64) AS {CAST(annotated_data_finite.inc_col AS Float64)|{annotated_data_finite.inc_col}}, annotated_data_finite.ts, annotated_data_finite.inc_col 08)--------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, min1@2 as min1, min2@3 as min2, max1@4 as max1, max2@5 as max2, count1@6 as count1, count2@7 as count2, avg1@8 as avg1, avg2@9 as avg2] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, min1@2 as min1, min2@3 as min2, max1@4 as max1, max2@5 as max2, count1@6 as count1, count2@7 as count2, avg1@8 as avg1, avg2@9 as avg2] 03)----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST], preserve_partitioning=[false] 04)------ProjectionExec: expr=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as sum1, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as max2, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@12 as count1, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@7 as count2, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@13 as avg1, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@8 as avg2, inc_col@3 as inc_col] 05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)), is_causal: false }], mode=[Sorted] @@ -2766,8 +2766,8 @@ logical_plan 06)----------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] 07)------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -01)ProjectionExec: expr=[first_value1@0 as first_value1, first_value2@1 as first_value2, last_value1@2 as last_value1, last_value2@3 as last_value2, nth_value1@4 as nth_value1] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[first_value1@0 as first_value1, first_value2@1 as first_value2, last_value1@2 as last_value1, last_value2@3 as last_value2, nth_value1@4 as nth_value1] 03)----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST], preserve_partitioning=[false] 04)------ProjectionExec: expr=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col] 05)--------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(2)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] @@ -2813,13 +2813,12 @@ logical_plan 07)------------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 08)--------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] -02)--GlobalLimitExec: skip=0, fetch=5 -03)----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] -04)------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)), is_causal: false }, 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)), is_causal: false }], mode=[Sorted] -05)--------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)), is_causal: false }, 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)), is_causal: false }], mode=[Sorted] -06)----------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] -07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +01)GlobalLimitExec: skip=0, fetch=5 +02)--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] +03)----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)), is_causal: false }, 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)), is_causal: false }], mode=[Sorted] +04)------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)), is_causal: false }, 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)), is_causal: false }], mode=[Sorted] +05)--------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] +06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT @@ -2860,14 +2859,12 @@ logical_plan 07)------------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 08)--------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] -02)--GlobalLimitExec: skip=0, fetch=5 -03)----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] -04)------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)), is_causal: false }, 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)), is_causal: false }], mode=[Sorted] -05)--------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)), is_causal: false }, 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)), is_causal: false }], mode=[Sorted] -06)----------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] -07)------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] - +01)GlobalLimitExec: skip=0, fetch=5 +02)--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] +03)----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)), is_causal: false }, 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)), is_causal: false }], mode=[Sorted] +04)------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)), is_causal: false }, 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)), is_causal: false }], mode=[Sorted] +05)--------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] +06)----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT @@ -2961,8 +2958,8 @@ logical_plan 09)----------------Projection: CAST(annotated_data_infinite2.c AS Int64) AS {CAST(annotated_data_infinite2.c AS Int64)|{annotated_data_infinite2.c}}, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d 10)------------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan -01)ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] 03)----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 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(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)), is_causal: true }], mode=[Linear] 04)------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 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(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[PartiallySorted([1, 0])] 05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 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(2)), end_bound: Following(UInt64(1)), is_causal: false }, SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite2.c) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] @@ -3363,8 +3360,8 @@ logical_plan 05)--------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 06)----------TableScan: aggregate_test_100 projection=[c3, c12] physical_plan -01)ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] 03)----SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST], preserve_partitioning=[false] 04)------ProjectionExec: expr=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max1, c3@0 as c3] 05)--------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Float64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] @@ -4066,8 +4063,8 @@ logical_plan 05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 06)----------TableScan: aggregate_test_100 projection=[c3, c4, c9] physical_plan -01)ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, sum1@3 as sum1] -02)--GlobalLimitExec: skip=0, fetch=5 +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, sum1@3 as sum1] 03)----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] 04)------ProjectionExec: expr=[c3@0 as c3, c4@1 as c4, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1] 05)--------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]